From 915dd2966084d78f7b4f3d482e6bd25f860c1e23 Mon Sep 17 00:00:00 2001 From: Alexey Goncharuk Date: Wed, 31 Jan 2018 11:22:26 +0300 Subject: [PATCH 001/314] IGNITE-7569 Fixed index rebuild future - Fixes #3454. --- .../GridDhtPartitionsExchangeFuture.java | 29 +++++++++- .../GridCacheDatabaseSharedManager.java | 57 ++++++++++++++----- 2 files changed, 69 insertions(+), 17 deletions(-) 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 6c09b6a4373df..a45c9b99cd85d 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 @@ -168,7 +168,11 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte /** */ private AtomicBoolean added = new AtomicBoolean(false); - /** Event latch. */ + /** + * Discovery event receive latch. There is a race between discovery event processing and single message + * processing, so it is possible to create an exchange future before the actual discovery event is received. + * This latch is notified when the discovery event arrives. + */ @GridToStringExclude private final CountDownLatch evtLatch = new CountDownLatch(1); @@ -344,6 +348,10 @@ public ExchangeContext context() { } /** + * Sets exchange actions associated with the exchange future (such as cache start or stop). + * Exchange actions is created from discovery event, so the actions must be set before the event is processed, + * thus the setter requires that {@code evtLatch} be armed. + * * @param exchActions Exchange actions. */ public void exchangeActions(ExchangeActions exchActions) { @@ -354,6 +362,20 @@ public void exchangeActions(ExchangeActions exchActions) { } /** + * Gets exchanges actions (such as cache start or stop) associated with the exchange future. + * Exchange actions can be {@code null} (for example, if the exchange is created for topology + * change event). + * + * @return Exchange actions. + */ + @Nullable public ExchangeActions exchangeActions() { + return exchActions; + } + + /** + * Sets affinity change message associated with the exchange. Affinity change message is required when + * centralized affinity change is performed. + * * @param affChangeMsg Affinity change message. */ public void affinityChangeMessage(CacheAffinityChangeMessage affChangeMsg) { @@ -361,9 +383,12 @@ public void affinityChangeMessage(CacheAffinityChangeMessage affChangeMsg) { } /** + * Gets the affinity topology version for which this exchange was created. If several exchanges + * were merged, initial version is the version of the earliest merged exchange. + * * @return Initial exchange version. */ - public AffinityTopologyVersion initialVersion() { + @Override public AffinityTopologyVersion initialVersion() { return exchId.topologyVersion(); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java index f833911135b7d..0b35f1830a912 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java @@ -105,6 +105,7 @@ import org.apache.ignite.internal.processors.cache.CacheGroupContext; import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor; import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor; +import org.apache.ignite.internal.processors.cache.ExchangeActions; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.StoredCacheData; @@ -320,8 +321,8 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan /** Thread local with buffers for the checkpoint threads. Each buffer represent one page for durable memory. */ private ThreadLocal threadBuf; - /** */ - private final ConcurrentMap idxRebuildFuts = new ConcurrentHashMap<>(); + /** Map from a cacheId to a future indicating that there is an in-progress index rebuild for the given cache. */ + private final ConcurrentMap> idxRebuildFuts = new ConcurrentHashMap<>(); /** * Lock holder for compatible folders mode. Null if lock holder was created at start node.
@@ -1127,33 +1128,59 @@ private void shutdownCheckpointer(boolean cancel) { // Before local node join event. if (clusterInTransitionStateToActive || (joinEvt && locNode && isSrvNode)) restoreState(); + + if (cctx.kernalContext().query().moduleEnabled()) { + ExchangeActions acts = fut.exchangeActions(); + + if (acts != null && !F.isEmpty(acts.cacheStartRequests())) { + for (ExchangeActions.CacheActionData actionData : acts.cacheStartRequests()) { + int cacheId = CU.cacheId(actionData.request().cacheName()); + + GridFutureAdapter old = idxRebuildFuts.put(cacheId, new GridFutureAdapter<>()); + + if (old != null) + old.onDone(); + } + } + } } /** {@inheritDoc} */ @Override public void rebuildIndexesIfNeeded(GridDhtPartitionsExchangeFuture fut) { if (cctx.kernalContext().query().moduleEnabled()) { for (final GridCacheContext cacheCtx : (Collection)cctx.cacheContexts()) { - if (cacheCtx.startTopologyVersion().equals(fut.initialVersion()) && - !cctx.pageStore().hasIndexStore(cacheCtx.groupId()) && cacheCtx.affinityNode()) { + if (cacheCtx.startTopologyVersion().equals(fut.initialVersion())) { final int cacheId = cacheCtx.cacheId(); + final GridFutureAdapter usrFut = idxRebuildFuts.get(cacheId); + + if (!cctx.pageStore().hasIndexStore(cacheCtx.groupId()) && cacheCtx.affinityNode()) { + IgniteInternalFuture rebuildFut = cctx.kernalContext().query() + .rebuildIndexesFromHash(Collections.singletonList(cacheCtx.cacheId())); - final IgniteInternalFuture rebuildFut = cctx.kernalContext().query() - .rebuildIndexesFromHash(Collections.singletonList(cacheCtx.cacheId())); + assert usrFut != null : "Missing user future for cache: " + cacheCtx.name(); - idxRebuildFuts.put(cacheId, rebuildFut); + rebuildFut.listen(new CI1() { + @Override public void apply(IgniteInternalFuture igniteInternalFut) { + idxRebuildFuts.remove(cacheId, usrFut); - rebuildFut.listen(new CI1() { - @Override public void apply(IgniteInternalFuture igniteInternalFut) { - idxRebuildFuts.remove(cacheId, rebuildFut); + usrFut.onDone(igniteInternalFut.error()); - CacheConfiguration ccfg = cacheCtx.config(); + CacheConfiguration ccfg = cacheCtx.config(); - if (ccfg != null) { - log().info("Finished indexes rebuilding for cache [name=" + ccfg.getName() - + ", grpName=" + ccfg.getGroupName() + ']'); + if (ccfg != null) { + log().info("Finished indexes rebuilding for cache [name=" + ccfg.getName() + + ", grpName=" + ccfg.getGroupName() + ']'); + } } + }); + } + else { + if (usrFut != null) { + idxRebuildFuts.remove(cacheId, usrFut); + + usrFut.onDone(); } - }); + } } } } From 8ea8609259039852ab0c26f26ac528c1ffae7c94 Mon Sep 17 00:00:00 2001 From: Alexey Goncharuk Date: Wed, 31 Jan 2018 11:24:57 +0300 Subject: [PATCH 002/314] IGNITE-7577 Fixing public API active flag on baseline changes - Fixes #3455. --- .../cluster/DiscoveryDataClusterState.java | 9 +- .../cluster/GridClusterStateProcessor.java | 2 +- .../CacheBaselineTopologyTest.java | 102 +++++++++++------- 3 files changed, 71 insertions(+), 42 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/DiscoveryDataClusterState.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/DiscoveryDataClusterState.java index 1c8e830710f8d..dea2ce79c311e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/DiscoveryDataClusterState.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/DiscoveryDataClusterState.java @@ -146,12 +146,19 @@ public UUID transitionRequestId() { } /** - * @return {@code True} if state change is in progress. + * @return {@code True} if any cluster state change is in progress (e.g. active state change, baseline change). */ public boolean transition() { return transitionReqId != null; } + /** + * @return {@code True} if cluster active state change is in progress, {@code false} otherwise. + */ + public boolean activeStateChanging() { + return transition() && active != prevState.active; + } + /** * @return State change exchange version. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java index f28df8ac14f80..aa23b61a3a653 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java @@ -180,7 +180,7 @@ public boolean compatibilityMode() { assert globalState != null; - if (globalState.transition()) { + if (globalState.transition() && globalState.activeStateChanging()) { Boolean transitionRes = globalState.transitionResult(); if (transitionRes != null) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBaselineTopologyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBaselineTopologyTest.java index e70773d205274..7b40b03cae393 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBaselineTopologyTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBaselineTopologyTest.java @@ -30,8 +30,6 @@ 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.cache.PartitionLossPolicy; import org.apache.ignite.cluster.BaselineNode; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.CacheConfiguration; @@ -40,9 +38,10 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.configuration.WALMode; import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology; -import org.apache.ignite.internal.util.lang.GridAbsPredicate; -import org.apache.ignite.internal.util.typedef.PA; +import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; @@ -85,6 +84,8 @@ public class CacheBaselineTopologyTest extends GridCommonAbstractTest { @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + cfg.setConsistentId(igniteInstanceName); + cfg.setDataStorageConfiguration( new DataStorageConfiguration().setDefaultDataRegionConfiguration( new DataRegionConfiguration() @@ -116,7 +117,7 @@ public void testTopologyChangesWithFixedBaseline() throws Exception { IgniteEx ignite = grid(0); - ignite.active(true); + ignite.cluster().active(true); awaitPartitionMapExchange(); @@ -257,6 +258,42 @@ public void testBaselineTopologyChangesFromClient() throws Exception { testBaselineTopologyChanges(true); } + /** + * @throws Exception if failed. + */ + public void testClusterActiveWhileBaselineChanging() throws Exception { + startGrids(NODE_COUNT); + + IgniteEx ig = grid(0); + + ig.cluster().active(true); + + assertTrue(ig.cluster().active()); + + startGrid(NODE_COUNT); + + IgniteInternalFuture fut = GridTestUtils.runAsync(() -> { + try { + U.sleep(100); + } + catch (IgniteInterruptedCheckedException e) { + e.printStackTrace(); + } + ig.cluster().setBaselineTopology(NODE_COUNT + 1); + }); + + while (!fut.isDone()) { + assertTrue(grid(0).cluster().active()); + assertTrue(grid(0).context().state().publicApiActiveState(false)); + assertTrue(grid(NODE_COUNT).cluster().active()); + assertTrue(grid(NODE_COUNT).context().state().publicApiActiveState(false)); + } + + assertNull(String.valueOf(fut.error()), fut.error()); + + assertEquals(NODE_COUNT + 1, ig.cluster().currentBaselineTopology().size()); + } + /** * @throws Exception If failed. */ @@ -275,7 +312,7 @@ private void testBaselineTopologyChanges(boolean fromClient) throws Exception { else ignite = grid(0); - ignite.active(true); + ignite.cluster().active(true); awaitPartitionMapExchange(); @@ -287,14 +324,13 @@ private void testBaselineTopologyChanges(boolean fromClient) throws Exception { nodes.put(ig.cluster().localNode(), ig); } - IgniteCache cache = - ignite.createCache( - new CacheConfiguration() - .setName(CACHE_NAME) - .setCacheMode(PARTITIONED) - .setBackups(1) - .setPartitionLossPolicy(READ_ONLY_SAFE) - ); + ignite.createCache( + new CacheConfiguration() + .setName(CACHE_NAME) + .setCacheMode(PARTITIONED) + .setBackups(1) + .setPartitionLossPolicy(READ_ONLY_SAFE) + ); int key = -1; @@ -307,8 +343,6 @@ private void testBaselineTopologyChanges(boolean fromClient) throws Exception { assert key >= 0; - int part = ignite.affinity(CACHE_NAME).partition(key); - Collection initialMapping = ignite.affinity(CACHE_NAME).mapKeyToPrimaryAndBackups(key); assert initialMapping.size() == 2 : initialMapping; @@ -474,7 +508,7 @@ public void testPrimaryLeftAndClusterRestart() throws Exception { IgniteEx ig = grid(0); - ig.active(true); + ig.cluster().active(true); IgniteCache cache = ig.createCache( @@ -544,14 +578,13 @@ else if (grid(i).localNode().equals(affNodes.get(1))) { primary = grid(primaryIdx); backup = grid(backupIdx); - boolean activated = GridTestUtils.waitForCondition(new GridAbsPredicate() { - @Override public boolean apply() { - for (int i = 0; i < NODE_COUNT; i++) - if (!grid(i).active()) - return false; - - return true; + boolean activated = GridTestUtils.waitForCondition(() -> { + for (int i = 0; i < NODE_COUNT; i++) { + if (!grid(i).cluster().active()) + return false; } + + return true; }, 10_000); assert activated; @@ -580,7 +613,7 @@ public void testMetadataUpdate() throws Exception { Ignite ignite3 = grid(3); - ignite3.active(true); + ignite3.cluster().active(true); CacheConfiguration repCacheCfg = new CacheConfiguration<>("replicated") .setCacheMode(CacheMode.REPLICATED) @@ -600,12 +633,7 @@ public void testMetadataUpdate() throws Exception { startGrids(5); - GridTestUtils.waitForCondition(new PA() { - @Override - public boolean apply() { - return grid(0).cluster().active(); - } - }, getTestTimeout()); + GridTestUtils.waitForCondition(() -> grid(0).cluster().active(), getTestTimeout()); for (int g = 0; g < 5; g++) { for (int i = 0; i < 100; i++) @@ -621,7 +649,7 @@ public void testClusterRestoredOnRestart() throws Exception { Ignite ignite3 = grid(3); - ignite3.active(true); + ignite3.cluster().active(true); stopGrid(0); @@ -640,12 +668,7 @@ public void testClusterRestoredOnRestart() throws Exception { startGrids(5); - GridTestUtils.waitForCondition(new PA() { - @Override - public boolean apply() { - return grid(0).cluster().active(); - } - }, getTestTimeout()); + GridTestUtils.waitForCondition(() -> grid(0).cluster().active(), getTestTimeout()); for (int g = 0; g < 5; g++) { for (int i = 0; i < 2048; i++) @@ -678,8 +701,7 @@ public void testNonPersistentCachesIgnoreBaselineTopology() throws Exception { private Collection baselineNodes(Collection clNodes) { Collection res = new ArrayList<>(clNodes.size()); - for (ClusterNode clN : clNodes) - res.add(clN); + res.addAll(clNodes); return res; } From c8ce1f66e98b3174d771a3b801a2538499dc2c3d Mon Sep 17 00:00:00 2001 From: Ivan Rakov Date: Wed, 31 Jan 2018 12:51:09 +0300 Subject: [PATCH 003/314] IGNITE-7475 Improved VerifyBackupPartitionsTask to calculate partition hashes in parallel - Fixes #3407. Signed-off-by: Alexey Goncharuk --- .../verify/VerifyBackupPartitionsTask.java | 157 +++++++++++++----- 1 file changed, 118 insertions(+), 39 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/VerifyBackupPartitionsTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/VerifyBackupPartitionsTask.java index 23aa0e1bac2a2..b884cb01ac934 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/VerifyBackupPartitionsTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/VerifyBackupPartitionsTask.java @@ -19,13 +19,22 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ForkJoinPool; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteInterruptedException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.compute.ComputeJob; @@ -162,6 +171,9 @@ public static class VerifyBackupPartitionsJob extends ComputeJobAdapter { /** Cache names. */ private Set cacheNames; + /** Counter of processed partitions. */ + private final AtomicInteger completionCntr = new AtomicInteger(0); + /** * @param names Names. */ @@ -208,7 +220,9 @@ private VerifyBackupPartitionsJob(Set names) { } } - Map res = new HashMap<>(); + List>> partHashCalcFutures = new ArrayList<>(); + + completionCntr.set(0); for (Integer grpId : grpIds) { CacheGroupContext grpCtx = ignite.context().cache().cacheGroup(grpId); @@ -218,62 +232,127 @@ private VerifyBackupPartitionsJob(Set names) { List parts = grpCtx.topology().localPartitions(); - for (GridDhtLocalPartition part : parts) { - if (!part.reserve()) - continue; + for (GridDhtLocalPartition part : parts) + partHashCalcFutures.add(calculatePartitionHashAsync(grpCtx, part)); + } - int partHash = 0; - long partSize; - long updateCntrBefore; + Map res = new HashMap<>(); - try { - if (part.state() != GridDhtPartitionState.OWNING) - continue; + long lastProgressLogTs = U.currentTimeMillis(); - updateCntrBefore = part.updateCounter(); + for (int i = 0; i < partHashCalcFutures.size(); ) { + Future> fut = partHashCalcFutures.get(i); - partSize = part.dataStore().fullSize(); + try { + Map partHash = fut.get(10, TimeUnit.SECONDS); - GridIterator it = grpCtx.offheap().partitionIterator(part.id()); + res.putAll(partHash); - while (it.hasNextX()) { - CacheDataRow row = it.nextX(); + i++; + } + catch (InterruptedException | ExecutionException e) { + for (int j = i + 1; j < partHashCalcFutures.size(); j++) + partHashCalcFutures.get(j).cancel(false); + + if (e instanceof InterruptedException) + throw new IgniteInterruptedException((InterruptedException)e); + else if (e.getCause() instanceof IgniteException) + throw (IgniteException)e.getCause(); + else + throw new IgniteException(e.getCause()); + } + catch (TimeoutException e) { + if (U.currentTimeMillis() - lastProgressLogTs > 3 * 60 * 1000L) { + lastProgressLogTs = U.currentTimeMillis(); - partHash += row.key().hashCode(); + log.warning("idle_verify is still running, processed " + completionCntr.get() + " of " + + partHashCalcFutures.size() + " local partitions"); + } + } + } - partHash += Arrays.hashCode(row.value().valueBytes(grpCtx.cacheObjectContext())); - } + return res; + } - long updateCntrAfter = part.updateCounter(); + /** + * @param grpCtx Group context. + * @param part Local partition. + */ + private Future> calculatePartitionHashAsync( + final CacheGroupContext grpCtx, + final GridDhtLocalPartition part + ) { + return ForkJoinPool.commonPool().submit(new Callable>() { + @Override public Map call() throws Exception { + return calculatePartitionHash(grpCtx, part); + } + }); + } - if (updateCntrBefore != updateCntrAfter) { - throw new IgniteException("Cluster is not idle: update counter of partition [grpId=" + - grpId + ", partId=" + part.id() + "] changed during hash calculation [before=" + - updateCntrBefore + ", after=" + updateCntrAfter + "]"); - } - } - catch (IgniteCheckedException e) { - U.error(log, "Can't calculate partition hash [grpId=" + grpId + - ", partId=" + part.id() + "]", e); - continue; - } - finally { - part.release(); - } + /** + * @param grpCtx Group context. + * @param part Local partition. + */ + private Map calculatePartitionHash( + CacheGroupContext grpCtx, + GridDhtLocalPartition part + ) { + if (!part.reserve()) + return Collections.emptyMap(); + + int partHash = 0; + long partSize; + long updateCntrBefore; + + try { + if (part.state() != GridDhtPartitionState.OWNING) + return Collections.emptyMap(); - Object consId = ignite.context().discovery().localNode().consistentId(); + updateCntrBefore = part.updateCounter(); - boolean isPrimary = part.primary(grpCtx.topology().readyTopologyVersion()); + partSize = part.dataStore().fullSize(); + + GridIterator it = grpCtx.offheap().partitionIterator(part.id()); + + while (it.hasNextX()) { + CacheDataRow row = it.nextX(); + + partHash += row.key().hashCode(); + + partHash += Arrays.hashCode(row.value().valueBytes(grpCtx.cacheObjectContext())); + } - PartitionKey partKey = new PartitionKey(grpId, part.id(), grpCtx.cacheOrGroupName()); + long updateCntrAfter = part.updateCounter(); - res.put(partKey, new PartitionHashRecord( - partKey, isPrimary, consId, partHash, updateCntrBefore, partSize)); + if (updateCntrBefore != updateCntrAfter) { + throw new IgniteException("Cluster is not idle: update counter of partition [grpId=" + + grpCtx.groupId() + ", partId=" + part.id() + "] changed during hash calculation [before=" + + updateCntrBefore + ", after=" + updateCntrAfter + "]"); } } + catch (IgniteCheckedException e) { + U.error(log, "Can't calculate partition hash [grpId=" + grpCtx.groupId() + + ", partId=" + part.id() + "]", e); - return res; + return Collections.emptyMap(); + } + finally { + part.release(); + } + + Object consId = ignite.context().discovery().localNode().consistentId(); + + boolean isPrimary = part.primary(grpCtx.topology().readyTopologyVersion()); + + PartitionKey partKey = new PartitionKey(grpCtx.groupId(), part.id(), grpCtx.cacheOrGroupName()); + + PartitionHashRecord partRec = new PartitionHashRecord( + partKey, isPrimary, consId, partHash, updateCntrBefore, partSize); + + completionCntr.incrementAndGet(); + + return Collections.singletonMap(partKey, partRec); } } From 258ff4299da20122d7c387cb8579264035c93c18 Mon Sep 17 00:00:00 2001 From: Alexey Goncharuk Date: Wed, 31 Jan 2018 16:52:24 +0300 Subject: [PATCH 004/314] IGNITE-7573 Fixed full API tests to be compliant with baseline topology --- .../cache/GridCacheAbstractFullApiSelfTest.java | 11 ++++------- .../IgniteBaselineAbstractFullApiSelfTest.java | 4 +++- 2 files changed, 7 insertions(+), 8 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 e6c9589a5ec30..2e6a19cb5903a 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 @@ -463,14 +463,11 @@ public void testSize() throws Exception { for (int i = 0; i < gridCount(); i++) assertEquals(globalPrimarySize, jcache(i).size(PRIMARY)); - int times = 1; + // Check how many instances of any given key there is in the cluster. + int globalSize = 0; - if (cacheMode() == REPLICATED) - times = gridCount(); - else if (cacheMode() == PARTITIONED) - times = Math.min(gridCount(), jcache().getConfiguration(CacheConfiguration.class).getBackups() + 1); - - int globalSize = globalPrimarySize * times; + for (String key : map.keySet()) + globalSize += affinity(jcache()).mapKeyToPrimaryAndBackups(key).size(); for (int i = 0; i < gridCount(); i++) assertEquals(globalSize, jcache(i).size(ALL)); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteBaselineAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteBaselineAbstractFullApiSelfTest.java index 8dcfc0b979926..d78c289f2e9e5 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteBaselineAbstractFullApiSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteBaselineAbstractFullApiSelfTest.java @@ -19,6 +19,7 @@ import org.apache.ignite.configuration.DataRegionConfiguration; import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.configuration.WALMode; import org.apache.ignite.internal.processors.cache.GridCacheAbstractFullApiSelfTest; /** @@ -33,7 +34,8 @@ public abstract class IgniteBaselineAbstractFullApiSelfTest extends GridCacheAbs .setDefaultDataRegionConfiguration( new DataRegionConfiguration() .setMaxSize(200 * 1024 * 1024) - .setPersistenceEnabled(true))); + .setPersistenceEnabled(true)) + .setWalMode(WALMode.LOG_ONLY)); return cfg; } From 254ed3a9c32d092702a0461509bf867cbd7cdee6 Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Thu, 1 Feb 2018 15:22:53 +0700 Subject: [PATCH 005/314] ignite-2.4.0 Update version. (cherry picked from commit 2e43749) --- modules/web-console/frontend/app/services/Version.service.js | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/web-console/frontend/app/services/Version.service.js b/modules/web-console/frontend/app/services/Version.service.js index 22d0732cff835..9bd9c7dfc302d 100644 --- a/modules/web-console/frontend/app/services/Version.service.js +++ b/modules/web-console/frontend/app/services/Version.service.js @@ -73,7 +73,7 @@ const compare = (a, b) => { export default class IgniteVersion { constructor() { - this.webConsole = '2.2.0'; + this.webConsole = '2.4.0'; this.supportedVersions = [ { From c1a9c0a404d77fba08170bedf14844f87abe3028 Mon Sep 17 00:00:00 2001 From: Alexey Goncharuk Date: Thu, 1 Feb 2018 13:17:28 +0300 Subject: [PATCH 006/314] IGNITE-7569 Fixing index rebuild future --- .../GridCacheDatabaseSharedManager.java | 32 ++++++++++++++----- 1 file changed, 24 insertions(+), 8 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java index 0b35f1830a912..5dc81c5e7ad48 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java @@ -71,6 +71,7 @@ import org.apache.ignite.configuration.DataRegionConfiguration; import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.configuration.NearCacheConfiguration; import org.apache.ignite.events.DiscoveryEvent; import org.apache.ignite.events.EventType; import org.apache.ignite.internal.GridKernalContext; @@ -1132,19 +1133,34 @@ private void shutdownCheckpointer(boolean cancel) { if (cctx.kernalContext().query().moduleEnabled()) { ExchangeActions acts = fut.exchangeActions(); - if (acts != null && !F.isEmpty(acts.cacheStartRequests())) { - for (ExchangeActions.CacheActionData actionData : acts.cacheStartRequests()) { - int cacheId = CU.cacheId(actionData.request().cacheName()); - - GridFutureAdapter old = idxRebuildFuts.put(cacheId, new GridFutureAdapter<>()); - - if (old != null) - old.onDone(); + if (acts != null) { + if (!F.isEmpty(acts.cacheStartRequests())) { + for (ExchangeActions.CacheActionData actionData : acts.cacheStartRequests()) + prepareIndexRebuildFuture(CU.cacheId(actionData.request().cacheName())); + } + else if (acts.localJoinContext() != null && !F.isEmpty(acts.localJoinContext().caches())) { + for (T2 tup : acts.localJoinContext().caches()) + prepareIndexRebuildFuture(tup.get1().cacheId()); } } } } + /** + * Creates a new index rebuild future that should be completed later after exchange is done. The future + * has to be created before exchange is initialized to guarantee that we will capture a correct future + * after activation or restore completes. + * If there was an old future for the given ID, it will be completed. + * + * @param cacheId Cache ID. + */ + private void prepareIndexRebuildFuture(int cacheId) { + GridFutureAdapter old = idxRebuildFuts.put(cacheId, new GridFutureAdapter<>()); + + if (old != null) + old.onDone(); + } + /** {@inheritDoc} */ @Override public void rebuildIndexesIfNeeded(GridDhtPartitionsExchangeFuture fut) { if (cctx.kernalContext().query().moduleEnabled()) { From e43799ce70cdbe03d9e206381d1d5138b820b075 Mon Sep 17 00:00:00 2001 From: Alexey Goncharuk Date: Thu, 1 Feb 2018 16:39:17 +0300 Subject: [PATCH 007/314] IGNITE-7520 Provide util-methods to get baseline from context - Fixes #3431. --- .../ignite/internal/util/IgniteUtils.java | 63 +++++++++++++++++++ 1 file changed, 63 insertions(+) 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 6e295dda48a7c..59430b0b9917b 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java @@ -194,7 +194,9 @@ 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.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cluster.BaselineTopology; import org.apache.ignite.internal.transactions.IgniteTxHeuristicCheckedException; import org.apache.ignite.internal.transactions.IgniteTxOptimisticCheckedException; import org.apache.ignite.internal.transactions.IgniteTxRollbackCheckedException; @@ -10294,6 +10296,67 @@ public static LockTracer lockTracer(Lock lock) { return new LockTracer(lock); } + /** + * @param ctx Context. + * + * @return instance of current baseline topology if it exists + */ + public static BaselineTopology getBaselineTopology(@NotNull GridKernalContext ctx) { + return ctx.state().clusterState().baselineTopology(); + } + + + /** + * @param cctx Context. + * + * @return instance of current baseline topology if it exists + */ + public static BaselineTopology getBaselineTopology(@NotNull GridCacheSharedContext cctx) { + return getBaselineTopology(cctx.kernalContext()); + } + + /** + * @param cctx Context. + * + * @return instance of current baseline topology if it exists + */ + public static BaselineTopology getBaselineTopology(@NotNull GridCacheContext cctx) { + return getBaselineTopology(cctx.kernalContext()); + } + + /** + * @param addr pointer in memory + * @param len how much byte to read (should divide 8) + * + * @return hex representation of memory region + */ + public static String toHexString(long addr, int len) { + assert (len & 0b111) == 0 && len > 0; + + StringBuilder sb = new StringBuilder(len * 2); + + for (int i = 0; i < len; i += 8) + sb.append(U.hexLong(GridUnsafe.getLong(addr + i))); + + return sb.toString(); + } + + /** + * @param buf which content should be converted to string + * + * @return hex representation of memory region + */ + public static String toHexString(ByteBuffer buf) { + assert (buf.capacity() & 0b111) == 0; + + StringBuilder sb = new StringBuilder(buf.capacity() * 2); + + for (int i = 0; i < buf.capacity(); i += 8) + sb.append(U.hexLong(buf.getLong(i))); + + return sb.toString(); + } + /** * */ From 8f5fc7cfb0624cf2048efad38dfff32f782116e8 Mon Sep 17 00:00:00 2001 From: Sergey Chugunov Date: Fri, 2 Feb 2018 11:24:29 +0300 Subject: [PATCH 008/314] IGNITE-7580 Fix compatibilityMode flag consistency This closes #3466 (cherry picked from commit 8f2045e) --- .../cluster/GridClusterStateProcessor.java | 31 +------------------ 1 file changed, 1 insertion(+), 30 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java index aa23b61a3a653..eaceb6923a230 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java @@ -65,7 +65,6 @@ 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.SB; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteCallable; import org.apache.ignite.lang.IgniteFuture; @@ -643,7 +642,7 @@ protected IgniteCheckedException concurrentStateChangeError(boolean activate) { BaselineTopologyHistory historyToSend = null; if (joiningNodeData != null) { - if (!joiningNodeData.hasJoiningNodeData()) { + if (!joiningNodeData.hasJoiningNodeData() || compatibilityMode) { //compatibility mode: old nodes don't send any data on join, so coordinator of new version //doesn't send BaselineTopology history, only its current globalState dataBag.addGridCommonData(STATE_PROC.ordinal(), globalState); @@ -791,40 +790,12 @@ private Collection baselineNodes() { return bltNodes; } - /** - * Verifies all nodes in current cluster topology support BaselineTopology feature - * so compatibilityMode flag is enabled to reset. - * - * @param discoCache - */ - private void verifyBaselineTopologySupport(DiscoCache discoCache) { - if (discoCache.minimumServerNodeVersion().compareTo(MIN_BLT_SUPPORTING_VER) < 0) { - SB sb = new SB("Cluster contains nodes that don't support BaselineTopology: ["); - - for (ClusterNode cn : discoCache.serverNodes()) { - if (cn.version().compareTo(MIN_BLT_SUPPORTING_VER) < 0) - sb - .a("[") - .a(cn.consistentId()) - .a(":") - .a(cn.version()) - .a("], "); - } - - sb.d(sb.length() - 2, sb.length()); - - throw new IgniteException(sb.a("]").toString()); - } - } - /** */ private IgniteInternalFuture changeGlobalState0(final boolean activate, BaselineTopology blt, boolean forceChangeBaselineTopology) { if (ctx.isDaemon() || ctx.clientNode()) { GridFutureAdapter fut = new GridFutureAdapter<>(); - verifyBaselineTopologySupport(ctx.discovery().discoCache()); - sendComputeChangeGlobalState(activate, blt, forceChangeBaselineTopology, fut); return fut; From d3ddd50cb2b889173176b6c47c9ff61410e1d909 Mon Sep 17 00:00:00 2001 From: Ilya Lantukh Date: Wed, 7 Feb 2018 13:33:28 +0300 Subject: [PATCH 009/314] IGNITE-7514 Affinity assignment should be recalculated when primary node is not OWNER (cherry picked from commit faf50f1) --- .../internal/events/DiscoveryCustomEvent.java | 34 ++++ .../cache/CacheAffinitySharedManager.java | 86 ++++++--- .../processors/cache/ClusterCachesInfo.java | 1 + .../GridCachePartitionExchangeManager.java | 2 +- .../dht/GridClientPartitionTopology.java | 9 +- .../dht/GridDhtLocalPartition.java | 3 - .../dht/GridDhtPartitionTopology.java | 10 +- .../dht/GridDhtPartitionTopologyImpl.java | 47 ++++- .../GridDhtPartitionsExchangeFuture.java | 144 +++++++++------ .../GridCacheDatabaseSharedManager.java | 5 + .../CacheBaselineTopologyTest.java | 169 +++++++++++++++++- 11 files changed, 424 insertions(+), 86 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/events/DiscoveryCustomEvent.java b/modules/core/src/main/java/org/apache/ignite/internal/events/DiscoveryCustomEvent.java index b3c6a2d865b2f..3b12b384f54be 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/events/DiscoveryCustomEvent.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/events/DiscoveryCustomEvent.java @@ -21,7 +21,10 @@ import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotDiscoveryMessage; +import org.apache.ignite.internal.processors.cluster.ChangeGlobalStateMessage; import org.apache.ignite.internal.util.typedef.internal.S; +import org.jetbrains.annotations.Nullable; /** * Custom event. @@ -85,4 +88,35 @@ public void affinityTopologyVersion(AffinityTopologyVersion affTopVer) { @Override public String toString() { return S.toString(DiscoveryCustomEvent.class, this, super.toString()); } + + /** + * @param evt Discovery event. + * @return {@code True} if event is DiscoveryCustomEvent that requires centralized affinity assignment. + */ + public static boolean requiresCentralizedAffinityAssignment(DiscoveryEvent evt) { + if (!(evt instanceof DiscoveryCustomEvent)) + return false; + + return requiresCentralizedAffinityAssignment(((DiscoveryCustomEvent)evt).customMessage()); + } + + /** + * @param msg Discovery custom message. + * @return {@code True} if message belongs to event that requires centralized affinity assignment. + */ + public static boolean requiresCentralizedAffinityAssignment(@Nullable DiscoveryCustomMessage msg) { + if (msg == null) + return false; + + if (msg instanceof ChangeGlobalStateMessage && ((ChangeGlobalStateMessage)msg).activate()) + return true; + + if (msg instanceof SnapshotDiscoveryMessage) { + SnapshotDiscoveryMessage snapMsg = (SnapshotDiscoveryMessage) msg; + + return snapMsg.needExchange() && snapMsg.needAssignPartitions(); + } + + return false; + } } \ No newline at end of file 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 4119f2357c896..7bf793c2f0b05 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 @@ -40,6 +40,7 @@ import org.apache.ignite.events.Event; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.cluster.ClusterTopologyServerNotFoundException; +import org.apache.ignite.internal.events.DiscoveryCustomEvent; import org.apache.ignite.internal.managers.discovery.DiscoCache; import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener; @@ -57,7 +58,6 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionMap; 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.cluster.ChangeGlobalStateFinishMessage; import org.apache.ignite.internal.processors.cluster.DiscoveryDataClusterState; import org.apache.ignite.internal.util.GridLongList; import org.apache.ignite.internal.util.GridPartitionStateMap; @@ -161,13 +161,15 @@ void onDiscoveryEvent(int type, ClusterNode node, AffinityTopologyVersion topVer, DiscoveryDataClusterState state) { - if (state.transition() || !state.active()) + if ((state.transition() || !state.active()) && + !DiscoveryCustomEvent.requiresCentralizedAffinityAssignment(customMsg)) return; if (type == EVT_NODE_JOINED && node.isLocal()) lastAffVer = null; - if (!CU.clientNode(node) && (type == EVT_NODE_FAILED || type == EVT_NODE_JOINED || type == EVT_NODE_LEFT)) { + if ((!CU.clientNode(node) && (type == EVT_NODE_FAILED || type == EVT_NODE_JOINED || type == EVT_NODE_LEFT)) || + DiscoveryCustomEvent.requiresCentralizedAffinityAssignment(customMsg)) { synchronized (mux) { assert lastAffVer == null || topVer.compareTo(lastAffVer) > 0; @@ -1260,10 +1262,12 @@ public GridAffinityAssignmentCache affinity(Integer grpId) { } /** + * Applies affinity diff from the received full message. + * * @param fut Current exchange future. * @param msg Finish exchange message. */ - public void mergeExchangesOnServerLeft(final GridDhtPartitionsExchangeFuture fut, + public void applyAffinityFromFullMessage(final GridDhtPartitionsExchangeFuture fut, final GridDhtPartitionsFullMessage msg) { final Map nodesByOrder = new HashMap<>(); @@ -1396,7 +1400,7 @@ public void onServerJoinWithExchangeMergeProtocol(GridDhtPartitionsExchangeFutur * @return Computed difference with ideal affinity. * @throws IgniteCheckedException If failed. */ - public Map onServerLeftWithExchangeMergeProtocol( + public Map onServerLeftWithExchangeMergeProtocol( final GridDhtPartitionsExchangeFuture fut) throws IgniteCheckedException { final ExchangeDiscoveryEvents evts = fut.context().events(); @@ -1404,6 +1408,36 @@ public Map onServerLeftWithExchangeMergePro assert fut.context().mergeExchanges(); assert evts.hasServerLeft(); + return onReassignmentEnforced(fut); + } + + /** + * Calculates affinity on coordinator for custom event types that require centralized assignment. + * + * @param fut Current exchange future. + * @return Computed difference with ideal affinity. + * @throws IgniteCheckedException If failed. + */ + public Map onCustomEventWithEnforcedAffinityReassignment( + final GridDhtPartitionsExchangeFuture fut) throws IgniteCheckedException + { + assert DiscoveryCustomEvent.requiresCentralizedAffinityAssignment(fut.firstEvent()); + + return onReassignmentEnforced(fut); + } + + /** + * Calculates new affinity assignment on coordinator and creates affinity diff messages for other nodes. + * + * @param fut Current exchange future. + * @return Computed difference with ideal affinity. + * @throws IgniteCheckedException If failed. + */ + private Map onReassignmentEnforced( + final GridDhtPartitionsExchangeFuture fut) throws IgniteCheckedException + { + final ExchangeDiscoveryEvents evts = fut.context().events(); + forAllRegisteredCacheGroups(new IgniteInClosureX() { @Override public void applyx(CacheGroupDescriptor desc) throws IgniteCheckedException { AffinityTopologyVersion topVer = evts.topologyVersion(); @@ -1418,7 +1452,7 @@ public Map onServerLeftWithExchangeMergePro } }); - Map>> diff = initAffinityOnNodeLeft0(evts.topologyVersion(), + Map>> diff = initAffinityBasedOnPartitionsAvailability(evts.topologyVersion(), fut, NODE_TO_ORDER, true); @@ -1642,17 +1676,16 @@ private GridDhtAffinityAssignmentResponse fetchAffinity(AffinityTopologyVersion } /** - * Called on exchange initiated by server node leave. + * Called on exchange initiated by server node leave or custom event with centralized affinity assignment. * * @param fut Exchange future. * @param crd Coordinator flag. * @throws IgniteCheckedException If failed. * @return {@code True} if affinity should be assigned by coordinator. */ - public boolean onServerLeft(final GridDhtPartitionsExchangeFuture fut, boolean crd) throws IgniteCheckedException { - ClusterNode leftNode = fut.firstEvent().eventNode(); - - assert !leftNode.isClient() : leftNode; + public boolean onCentralizedAffinityChange(final GridDhtPartitionsExchangeFuture fut, boolean crd) throws IgniteCheckedException { + assert (fut.events().hasServerLeft() && !fut.firstEvent().eventNode().isClient()) || + DiscoveryCustomEvent.requiresCentralizedAffinityAssignment(fut.firstEvent()) : fut.firstEvent(); if (crd) { // Need initialize CacheGroupHolders if this node become coordinator on this exchange. @@ -2066,7 +2099,7 @@ public IgniteInternalFuture>>> initAffinity initFut.listen(new IgniteInClosure>() { @Override public void apply(IgniteInternalFuture initFut) { try { - resFut.onDone(initAffinityOnNodeLeft0(fut.initialVersion(), fut, NODE_TO_ID, false)); + resFut.onDone(initAffinityBasedOnPartitionsAvailability(fut.initialVersion(), fut, NODE_TO_ID, false)); } catch (IgniteCheckedException e) { resFut.onDone(e); @@ -2077,10 +2110,13 @@ public IgniteInternalFuture>>> initAffinity return resFut; } else - return new GridFinishedFuture<>(initAffinityOnNodeLeft0(fut.initialVersion(), fut, NODE_TO_ID, false)); + return new GridFinishedFuture<>(initAffinityBasedOnPartitionsAvailability(fut.initialVersion(), fut, NODE_TO_ID, false)); } /** + * Initializes current affinity assignment based on partitions availability. + * Nodes that have most recent data will be considered affinity nodes. + * * @param topVer Topology version. * @param fut Exchange future. * @param c Closure converting affinity diff. @@ -2088,12 +2124,17 @@ public IgniteInternalFuture>>> initAffinity * @return Affinity assignment. * @throws IgniteCheckedException If failed. */ - private Map>> initAffinityOnNodeLeft0(final AffinityTopologyVersion topVer, + private Map>> initAffinityBasedOnPartitionsAvailability(final AffinityTopologyVersion topVer, final GridDhtPartitionsExchangeFuture fut, final IgniteClosure c, final boolean initAff) throws IgniteCheckedException { - final WaitRebalanceInfo waitRebalanceInfo = new WaitRebalanceInfo(fut.context().events().lastServerEventVersion()); + final boolean enforcedCentralizedAssignment = + DiscoveryCustomEvent.requiresCentralizedAffinityAssignment(fut.firstEvent()); + + final WaitRebalanceInfo waitRebalanceInfo = enforcedCentralizedAssignment ? + new WaitRebalanceInfo(fut.exchangeId().topologyVersion()) : + new WaitRebalanceInfo(fut.context().events().lastServerEventVersion()); final Collection aliveNodes = fut.context().events().discoveryCache().serverNodes(); @@ -2103,13 +2144,14 @@ private Map>> initAffinityOnNodeLeft0(final Af @Override public void applyx(CacheGroupDescriptor desc) throws IgniteCheckedException { CacheGroupHolder grpHolder = groupHolder(topVer, desc); - if (!grpHolder.rebalanceEnabled || fut.cacheGroupAddedOnExchange(desc.groupId(), desc.receivedFrom())) + if (!grpHolder.rebalanceEnabled || + (fut.cacheGroupAddedOnExchange(desc.groupId(), desc.receivedFrom()) && !enforcedCentralizedAssignment)) return; AffinityTopologyVersion affTopVer = grpHolder.affinity().lastVersion(); - assert affTopVer.topologyVersion() > 0 && !affTopVer.equals(topVer) : "Invalid affinity version " + - "[last=" + affTopVer + ", futVer=" + topVer + ", grp=" + desc.cacheOrGroupName() + ']'; + assert (affTopVer.topologyVersion() > 0 && !affTopVer.equals(topVer)) || enforcedCentralizedAssignment : + "Invalid affinity version [last=" + affTopVer + ", futVer=" + topVer + ", grp=" + desc.cacheOrGroupName() + ']'; List> curAssignment = grpHolder.affinity().assignments(affTopVer); List> newAssignment = grpHolder.affinity().idealAssignment(); @@ -2141,6 +2183,12 @@ private Map>> initAffinityOnNodeLeft0(final Af ", node=" + newPrimary + ", topVer=" + topVer + ']'; + List owners = top.owners(p); + + // It is essential that curPrimary node has partition in OWNING state. + if (!owners.isEmpty() && !owners.contains(curPrimary)) + curPrimary = owners.get(0); + if (curPrimary != null && newPrimary != null && !curPrimary.equals(newPrimary)) { if (aliveNodes.contains(curPrimary)) { GridDhtPartitionState state = top.partitionState(newPrimary.id(), p); @@ -2173,8 +2221,6 @@ private Map>> initAffinityOnNodeLeft0(final Af } if (newNodes0 == null) { - List owners = top.owners(p); - for (ClusterNode owner : owners) { if (aliveNodes.contains(owner)) { newNodes0 = latePrimaryAssignment(grpHolder.affinity(), diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java index 08a910b81603d..2b2fb559c182e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java @@ -1186,6 +1186,7 @@ public void onStateChangeFinish(ChangeGlobalStateFinishMessage msg) { /** * @param msg Message. * @param topVer Current topology version. + * @param curState Current cluster state. * @return Exchange action. * @throws IgniteCheckedException If configuration validation failed. */ 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 9b9284f04b10a..8aa6db933a5d0 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 @@ -965,7 +965,7 @@ public void scheduleResendPartitions() { * For coordinator causes {@link GridDhtPartitionsFullMessage FullMessages} send, * for non coordinator - {@link GridDhtPartitionsSingleMessage SingleMessages} send */ - private void refreshPartitions() { + public void refreshPartitions() { // TODO https://issues.apache.org/jira/browse/IGNITE-6857 if (cctx.snapshot().snapshotOperationInProgress()) { scheduleResendPartitions(); 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 e994113087ec8..def00f3f9ae89 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 @@ -256,9 +256,9 @@ private String mapString(GridDhtPartitionMap map) { } /** {@inheritDoc} */ - @Override public void initPartitionsWhenAffinityReady(AffinityTopologyVersion affVer, + @Override public boolean initPartitionsWhenAffinityReady(AffinityTopologyVersion affVer, GridDhtPartitionsExchangeFuture exchFut) { - // No-op. + return false; } /** {@inheritDoc} */ @@ -382,6 +382,11 @@ else if (!node2part.nodeId().equals(loc.id())) { fullMapString() + ']'); } + /** {@inheritDoc} */ + @Override public void afterStateRestored(AffinityTopologyVersion topVer) { + // no-op + } + /** {@inheritDoc} */ @Override public boolean afterExchange(GridDhtPartitionsExchangeFuture exchFut) throws IgniteCheckedException { AffinityTopologyVersion topVer = exchFut.topologyVersion(); 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 e1f1d6f43ce1e..e63aab6c0dc78 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 @@ -213,9 +213,6 @@ public class GridDhtLocalPartition extends GridCacheConcurrentMapImpl implements // TODO ignite-db throw new IgniteException(e); } - - // Todo log moving state - casState(state.get(), MOVING); } /** 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 4ae68ef739f8c..13564c2af2666 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 @@ -117,11 +117,19 @@ public void beforeExchange(GridDhtPartitionsExchangeFuture exchFut, /** * @param affVer Affinity version. * @param exchFut Exchange future. + * @return {@code True} if partitions must be refreshed. * @throws IgniteInterruptedCheckedException If interrupted. */ - public void initPartitionsWhenAffinityReady(AffinityTopologyVersion affVer, GridDhtPartitionsExchangeFuture exchFut) + public boolean initPartitionsWhenAffinityReady(AffinityTopologyVersion affVer, GridDhtPartitionsExchangeFuture exchFut) throws IgniteInterruptedCheckedException; + /** + * Initializes local data structures after partitions are restored from persistence. + * + * @param topVer Topology version. + */ + public void afterStateRestored(AffinityTopologyVersion topVer); + /** * Post-initializes this topology. * 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 0a2c1541bd70d..020c3e7ad34bb 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 @@ -299,10 +299,12 @@ private String mapString(GridDhtPartitionMap map) { } /** {@inheritDoc} */ - @Override public void initPartitionsWhenAffinityReady(AffinityTopologyVersion affVer, + @Override public boolean initPartitionsWhenAffinityReady(AffinityTopologyVersion affVer, GridDhtPartitionsExchangeFuture exchFut) throws IgniteInterruptedCheckedException { + boolean needRefresh; + ctx.database().checkpointReadLock(); try { @@ -310,11 +312,11 @@ private String mapString(GridDhtPartitionMap map) { try { if (stopping) - return; + return false; long updateSeq = this.updateSeq.incrementAndGet(); - initPartitions0(affVer, exchFut, updateSeq); + needRefresh = initPartitions0(affVer, exchFut, updateSeq); consistencyCheck(); } @@ -325,16 +327,21 @@ private String mapString(GridDhtPartitionMap map) { finally { ctx.database().checkpointReadUnlock(); } + + return needRefresh; } /** * @param affVer Affinity version to use. * @param exchFut Exchange future. * @param updateSeq Update sequence. + * @return {@code True} if partitions must be refreshed. */ - private void initPartitions0(AffinityTopologyVersion affVer, GridDhtPartitionsExchangeFuture exchFut, long updateSeq) { + private boolean initPartitions0(AffinityTopologyVersion affVer, GridDhtPartitionsExchangeFuture exchFut, long updateSeq) { List> aff = grp.affinity().readyAssignments(affVer); + boolean needRefresh = false; + if (grp.affinityNode()) { ClusterNode loc = ctx.localNode(); @@ -378,6 +385,8 @@ private void initPartitions0(AffinityTopologyVersion affVer, GridDhtPartitionsEx ", part=" + locPart + ']'); } + needRefresh = true; + updateSeq = updateLocal(p, locPart.state(), updateSeq, affVer); } } @@ -423,6 +432,8 @@ else if (belongs) { } updateRebalanceVersion(aff); + + return needRefresh; } /** @@ -616,6 +627,30 @@ private boolean partitionLocalNode(int p, AffinityTopologyVersion topVer) { return grp.affinity().nodes(p, topVer).contains(ctx.localNode()); } + /** {@inheritDoc} */ + @Override public void afterStateRestored(AffinityTopologyVersion topVer) { + lock.writeLock().lock(); + + try { + if (node2part == null) + return; + + long updateSeq = this.updateSeq.incrementAndGet(); + + for (int p = 0; p < grp.affinity().partitions(); p++) { + GridDhtLocalPartition locPart = locParts.get(p); + + if (locPart == null) + updateLocal(p, EVICTED, updateSeq, topVer); + else + updateLocal(p, locPart.state(), updateSeq, topVer); + } + } + finally { + lock.writeLock().unlock(); + } + } + /** {@inheritDoc} */ @Override public boolean afterExchange(GridDhtPartitionsExchangeFuture exchFut) { boolean changed = false; @@ -996,9 +1031,11 @@ else if (loc != null && state == RENTING && !showRenting) { map.put(i, part.state()); } + GridDhtPartitionMap locPartMap = node2part != null ? node2part.get(ctx.localNodeId()) : null; + return new GridDhtPartitionMap(ctx.localNodeId(), updateSeq.get(), - readyTopVer, + locPartMap != null ? locPartMap.topologyVersion() : readyTopVer, map, true); } 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 a45c9b99cd85d..695c840de302a 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 @@ -130,6 +130,9 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte private static final int RELEASE_FUTURE_DUMP_THRESHOLD = IgniteSystemProperties.getInteger(IGNITE_PARTITION_RELEASE_FUTURE_DUMP_THRESHOLD, 0); + /** */ + private static final IgniteProductVersion FORCE_AFF_REASSIGNMENT_SINCE = IgniteProductVersion.fromString("2.4.3"); + /** */ @GridToStringExclude private final Object mux = new Object(); @@ -231,6 +234,12 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte */ private boolean centralizedAff; + /** + * Enforce affinity reassignment based on actual partition distribution. This mode should be used when partitions + * might be distributed not according to affinity assignment. + */ + private boolean forceAffReassignment; + /** Change global state exception. */ private Exception changeGlobalStateE; @@ -615,6 +624,9 @@ public void init(boolean newCrd) throws IgniteInterruptedCheckedException { DiscoveryCustomMessage msg = ((DiscoveryCustomEvent)firstDiscoEvt).customMessage(); + forceAffReassignment = DiscoveryCustomEvent.requiresCentralizedAffinityAssignment(msg) + && firstEventCache().minimumNodeVersion().compareToIgnoreTimestamp(FORCE_AFF_REASSIGNMENT_SINCE) >= 0; + if (msg instanceof ChangeGlobalStateMessage) { assert exchActions != null && !exchActions.empty(); @@ -636,6 +648,9 @@ else if (msg instanceof WalStateAbstractMessage) exchange = onAffinityChangeRequest(crdNode); } + if (forceAffReassignment) + cctx.affinity().onCentralizedAffinityChange(this, crdNode); + initCoordinatorCaches(newCrd); } else { @@ -781,7 +796,7 @@ private void initTopologies() throws IgniteCheckedException { if (grp.isLocal()) continue; - grp.topology().beforeExchange(this, !centralizedAff, false); + grp.topology().beforeExchange(this, !centralizedAff && !forceAffReassignment, false); } } } @@ -924,8 +939,6 @@ private ExchangeType onClusterStateChangeRequest(boolean crd) { else if (req.activate()) { // TODO: BLT changes on inactive cluster can't be handled easily because persistent storage hasn't been initialized yet. try { - cctx.affinity().onBaselineTopologyChanged(this, crd); - if (CU.isPersistenceEnabled(cctx.kernalContext().config()) && !cctx.kernalContext().clientNode()) cctx.kernalContext().state().onBaselineTopologyChanged(req.baselineTopology(), req.prevBaselineTopologyHistoryItem()); @@ -962,7 +975,8 @@ private ExchangeType onCacheChangeRequest(boolean crd) throws IgniteCheckedExcep * @return Exchange type. */ private ExchangeType onCustomMessageNoAffinityChange(boolean crd) { - cctx.affinity().onCustomMessageNoAffinityChange(this, crd, exchActions); + if (!forceAffReassignment) + cctx.affinity().onCustomMessageNoAffinityChange(this, crd, exchActions); return cctx.kernalContext().clientNode() ? ExchangeType.CLIENT : ExchangeType.ALL; } @@ -1017,7 +1031,7 @@ private ExchangeType onServerNodeEvent(boolean crd) throws IgniteCheckedExceptio exchCtx.events().warnNoAffinityNodes(cctx); - centralizedAff = cctx.affinity().onServerLeft(this, crd); + centralizedAff = cctx.affinity().onCentralizedAffinityChange(this, crd); } else cctx.affinity().onServerJoin(this, crd); @@ -1088,8 +1102,6 @@ private void distributedExchange() throws IgniteCheckedException { } } - cctx.database().beforeExchange(this); - if (!exchCtx.mergeExchanges()) { for (CacheGroupContext grp : cctx.cache().cacheGroups()) { if (grp.isLocal() || cacheGroupStopping(grp.groupId())) @@ -1097,10 +1109,14 @@ private void distributedExchange() throws IgniteCheckedException { // It is possible affinity is not initialized yet if node joins to cluster. if (grp.affinity().lastVersion().topologyVersion() > 0) - grp.topology().beforeExchange(this, !centralizedAff, false); + grp.topology().beforeExchange(this, !centralizedAff && !forceAffReassignment, false); } } + // It is necessary to run database callback after all topology callbacks, so partition states could be + // correctly restored from the persistent store. + cctx.database().beforeExchange(this); + changeWalModeIfNeeded(); if (crd.isLocal()) { @@ -1534,19 +1550,24 @@ public void finishMerged() { } if (err == null) { - if (centralizedAff) { + if (centralizedAff || forceAffReassignment) { assert !exchCtx.mergeExchanges(); for (CacheGroupContext grp : cctx.cache().cacheGroups()) { if (grp.isLocal()) continue; + boolean needRefresh = false; + try { - grp.topology().initPartitionsWhenAffinityReady(res, this); + needRefresh = grp.topology().initPartitionsWhenAffinityReady(res, this); } catch (IgniteInterruptedCheckedException e) { U.error(log, "Failed to initialize partitions.", e); } + + if (needRefresh) + cctx.exchange().refreshPartitions(); } } @@ -2342,7 +2363,7 @@ private void onAllReceived(@Nullable Collection sndResNodes) { if (!exchCtx.mergeExchanges() && !crd.equals(events().discoveryCache().serverNodes().get(0))) { for (CacheGroupContext grp : cctx.cache().cacheGroups()) { if (!grp.isLocal()) - grp.topology().beforeExchange(this, !centralizedAff, false); + grp.topology().beforeExchange(this, !centralizedAff && !forceAffReassignment, false); } } @@ -2474,6 +2495,9 @@ else if (discoveryCustomMessage instanceof SnapshotDiscoveryMessage detectLostPartitions(resTopVer); } + if (!exchCtx.mergeExchanges() && forceAffReassignment) + idealAffDiff = cctx.affinity().onCustomEventWithEnforcedAffinityReassignment(this); + for (CacheGroupContext grpCtx : cctx.cache().cacheGroups()) { if (!grpCtx.isLocal()) grpCtx.topology().applyUpdateCounters(); @@ -2496,6 +2520,8 @@ else if (discoveryCustomMessage instanceof SnapshotDiscoveryMessage if (exchCtx.events().hasServerLeft()) msg.idealAffinityDiff(idealAffDiff); } + else if (forceAffReassignment) + msg.idealAffinityDiff(idealAffDiff); msg.prepareMarshal(cctx); @@ -2549,65 +2575,69 @@ else if (discoveryCustomMessage instanceof SnapshotDiscoveryMessage nodes.addAll(sndResNodes); } - IgniteCheckedException err = null; + if (!nodes.isEmpty()) + sendAllPartitions(msg, nodes, mergedJoinExchMsgs0, joinedNodeAff); - if (stateChangeExchange()) { - StateChangeRequest req = exchActions.stateChangeRequest(); + if (!stateChangeExchange()) + onDone(exchCtx.events().topologyVersion(), null); - assert req != null : exchActions; + for (Map.Entry e : pendingSingleMsgs.entrySet()) { + if (log.isInfoEnabled()) { + log.info("Process pending message on coordinator [node=" + e.getKey() + + ", ver=" + initialVersion() + + ", resVer=" + resTopVer + ']'); + } - boolean stateChangeErr = false; + processSingleMessage(e.getKey(), e.getValue()); + } + } - if (!F.isEmpty(changeGlobalStateExceptions)) { - stateChangeErr = true; + if (stateChangeExchange()) { + IgniteCheckedException err = null; - err = new IgniteCheckedException("Cluster state change failed."); + StateChangeRequest req = exchActions.stateChangeRequest(); - cctx.kernalContext().state().onStateChangeError(changeGlobalStateExceptions, req); - } - else { - boolean hasMoving = !partsToReload.isEmpty(); + assert req != null : exchActions; - Set waitGrps = cctx.affinity().waitGroups(); + boolean stateChangeErr = false; - if (!hasMoving) { - for (CacheGroupContext grpCtx : cctx.cache().cacheGroups()) { - if (waitGrps.contains(grpCtx.groupId()) && grpCtx.topology().hasMovingPartitions()) { - hasMoving = true; + if (!F.isEmpty(changeGlobalStateExceptions)) { + stateChangeErr = true; - break; - } + err = new IgniteCheckedException("Cluster state change failed."); - } - } + cctx.kernalContext().state().onStateChangeError(changeGlobalStateExceptions, req); + } + else { + boolean hasMoving = !partsToReload.isEmpty(); - cctx.kernalContext().state().onExchangeFinishedOnCoordinator(this, hasMoving); - } + Set waitGrps = cctx.affinity().waitGroups(); - boolean active = !stateChangeErr && req.activate(); + if (!hasMoving) { + for (CacheGroupContext grpCtx : cctx.cache().cacheGroups()) { + if (waitGrps.contains(grpCtx.groupId()) && grpCtx.topology().hasMovingPartitions()) { + hasMoving = true; - ChangeGlobalStateFinishMessage stateFinishMsg = new ChangeGlobalStateFinishMessage( - req.requestId(), - active, - !stateChangeErr); + break; + } - cctx.discovery().sendCustomEvent(stateFinishMsg); + } + } + + cctx.kernalContext().state().onExchangeFinishedOnCoordinator(this, hasMoving); } - if (!nodes.isEmpty()) - sendAllPartitions(msg, nodes, mergedJoinExchMsgs0, joinedNodeAff); + boolean active = !stateChangeErr && req.activate(); - onDone(exchCtx.events().topologyVersion(), err); + ChangeGlobalStateFinishMessage stateFinishMsg = new ChangeGlobalStateFinishMessage( + req.requestId(), + active, + !stateChangeErr); - for (Map.Entry e : pendingSingleMsgs.entrySet()) { - if (log.isInfoEnabled()) { - log.info("Process pending message on coordinator [node=" + e.getKey() + - ", ver=" + initialVersion() + - ", resVer=" + resTopVer + ']'); - } + cctx.discovery().sendCustomEvent(stateFinishMsg); - processSingleMessage(e.getKey(), e.getValue()); - } + if (!centralizedAff) + onDone(exchCtx.events().topologyVersion(), err); } } catch (IgniteCheckedException e) { @@ -2950,7 +2980,7 @@ private void processFullMessage(boolean checkCrd, ClusterNode node, GridDhtParti cctx.affinity().onLocalJoin(this, msg, resTopVer); else { if (exchCtx.events().hasServerLeft()) - cctx.affinity().mergeExchangesOnServerLeft(this, msg); + cctx.affinity().applyAffinityFromFullMessage(this, msg); else cctx.affinity().onServerJoinWithExchangeMergeProtocol(this, false); @@ -2964,6 +2994,8 @@ private void processFullMessage(boolean checkCrd, ClusterNode node, GridDhtParti } else if (localJoinExchange() && !exchCtx.fetchAffinityOnJoin()) cctx.affinity().onLocalJoin(this, msg, resTopVer); + else if (forceAffReassignment) + cctx.affinity().applyAffinityFromFullMessage(this, msg); updatePartitionFullMap(resTopVer, msg); @@ -3073,6 +3105,9 @@ public void onAffinityChangeMessage(final ClusterNode node, final CacheAffinityC crd.isLocal(), msg); + IgniteCheckedException err = !F.isEmpty(msg.partitionsMessage().getErrorsMap()) ? + new IgniteCheckedException("Cluster state change failed.") : null; + if (!crd.isLocal()) { GridDhtPartitionsFullMessage partsMsg = msg.partitionsMessage(); @@ -3080,9 +3115,12 @@ public void onAffinityChangeMessage(final ClusterNode node, final CacheAffinityC assert partsMsg.lastVersion() != null : partsMsg; updatePartitionFullMap(resTopVer, partsMsg); + + if (exchActions != null && exchActions.stateChangeRequest() != null && err != null) + cctx.kernalContext().state().onStateChangeError(msg.partitionsMessage().getErrorsMap(), exchActions.stateChangeRequest()); } - onDone(resTopVer); + onDone(resTopVer, err); } else { if (log.isDebugEnabled()) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java index 5dc81c5e7ad48..f1cc4ce562574 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java @@ -112,6 +112,8 @@ import org.apache.ignite.internal.processors.cache.StoredCacheData; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition; 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.GridDhtPartitionTopologyImpl; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; import org.apache.ignite.internal.processors.cache.persistence.file.FileIO; import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStore; @@ -2344,6 +2346,9 @@ else if (restore != null) { updateState(part, restore.get1()); } } + + // After partition states are restored, it is necessary to update internal data structures in topology. + grp.topology().afterStateRestored(grp.topology().lastTopologyChangeVersion()); } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBaselineTopologyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBaselineTopologyTest.java index 7b40b03cae393..6ccb450319180 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBaselineTopologyTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBaselineTopologyTest.java @@ -25,11 +25,16 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Random; import java.util.Set; +import java.util.UUID; 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.AffinityFunction; +import org.apache.ignite.cache.affinity.AffinityFunctionContext; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; import org.apache.ignite.cluster.BaselineNode; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.CacheConfiguration; @@ -40,8 +45,15 @@ import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.TestDelayingCommunicationSpi; +import org.apache.ignite.internal.managers.communication.GridIoMessage; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemandMessage; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionFullMap; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessage; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; @@ -62,6 +74,9 @@ public class CacheBaselineTopologyTest extends GridCommonAbstractTest { /** */ private static final int NODE_COUNT = 4; + /** */ + private static boolean delayRebalance = false; + /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { super.beforeTest(); @@ -106,6 +121,9 @@ public class CacheBaselineTopologyTest extends GridCommonAbstractTest { if (client) cfg.setClientMode(true); + if (delayRebalance) + cfg.setCommunicationSpi(new DelayRebalanceCommunicationSpi()); + return cfg; } @@ -594,7 +612,10 @@ else if (grid(i).localNode().equals(affNodes.get(1))) { assertEquals(val2, primary.cache(CACHE_NAME).get(key)); assertEquals(val2, backup.cache(CACHE_NAME).get(key)); - primary.cache(CACHE_NAME).rebalance().get(); + for (int i = 0; i < NODE_COUNT; i++) + grid(i).cache(CACHE_NAME).rebalance().get(); + + awaitPartitionMapExchange(); affNodes = (List) ig.affinity(CACHE_NAME).mapKeyToPrimaryAndBackups(key); @@ -697,6 +718,83 @@ public void testNonPersistentCachesIgnoreBaselineTopology() throws Exception { assertTrue(ig.affinity(inMemoryCache.getName()).allPartitions(newNode.cluster().localNode()).length > 0); } + /** + * @throws Exception if failed. + */ + public void testAffinityAssignmentChangedAfterRestart() throws Exception { + delayRebalance = false; + + int parts = 32; + + final List partMapping = new ArrayList<>(); + + for (int p = 0; p < parts; p++) + partMapping.add(p); + + final AffinityFunction affFunc = new TestAffinityFunction(new RendezvousAffinityFunction(false, parts)); + + TestAffinityFunction.partsAffMapping = partMapping; + + String cacheName = CACHE_NAME + 2; + + startGrids(4); + + IgniteEx ig = grid(0); + + ig.cluster().active(true); + + IgniteCache cache = ig.createCache( + new CacheConfiguration() + .setName(cacheName) + .setCacheMode(PARTITIONED) + .setBackups(1) + .setPartitionLossPolicy(READ_ONLY_SAFE) + .setReadFromBackup(true) + .setWriteSynchronizationMode(FULL_SYNC) + .setRebalanceDelay(-1) + .setAffinity(affFunc)); + + Map keyToConsId = new HashMap<>(); + + for (int k = 0; k < 1000; k++) { + cache.put(k, k); + + keyToConsId.put(k, ig.affinity(cacheName).mapKeyToNode(k).consistentId().toString()); + } + + stopAllGrids(); + + Collections.shuffle(TestAffinityFunction.partsAffMapping, new Random(1)); + + delayRebalance = true; + + startGrids(4); + + ig = grid(0); + + ig.active(true); + + cache = ig.cache(cacheName); + + GridDhtPartitionFullMap partMap = ig.cachex(cacheName).context().topology().partitionMap(false); + + for (int i = 1; i < 4; i++) { + IgniteEx ig0 = grid(i); + + for (int p = 0; p < 32; p++) + assertEqualsCollections(ig.affinity(cacheName).mapPartitionToPrimaryAndBackups(p), ig0.affinity(cacheName).mapPartitionToPrimaryAndBackups(p)); + } + + for (Map.Entry e : keyToConsId.entrySet()) { + int p = ig.affinity(cacheName).partition(e.getKey()); + + assertEquals("p=" + p, GridDhtPartitionState.OWNING, partMap.get(ig.affinity(cacheName).mapKeyToNode(e.getKey()).id()).get(p)); + } + + for (int k = 0; k < 1000; k++) + assertEquals("k=" + k, Integer.valueOf(k), cache.get(k)); + } + /** */ private Collection baselineNodes(Collection clNodes) { Collection res = new ArrayList<>(clNodes.size()); @@ -758,4 +856,73 @@ private TestValue(int a) { return result; } } + + /** + * + */ + private static class TestAffinityFunction implements AffinityFunction { + /** */ + private final AffinityFunction delegate; + + /** */ + private static List partsAffMapping; + + /** */ + public TestAffinityFunction(AffinityFunction delegate) { + this.delegate = delegate; + } + + /** {@inheritDoc} */ + @Override public void reset() { + delegate.reset();; + } + + /** {@inheritDoc} */ + @Override public int partitions() { + return delegate.partitions(); + } + + /** {@inheritDoc} */ + @Override public int partition(Object key) { + return delegate.partition(key); + } + + /** {@inheritDoc} */ + @Override public List> assignPartitions(AffinityFunctionContext affCtx) { + List> res0 = delegate.assignPartitions(affCtx); + + List> res = new ArrayList<>(res0.size()); + + for (int p = 0; p < res0.size(); p++) + res.add(p, null); + + for (int p = 0; p < res0.size(); p++) + res.set(partsAffMapping.get(p), res0.get(p)); + + return res; + } + + /** {@inheritDoc} */ + @Override public void removeNode(UUID nodeId) { + delegate.removeNode(nodeId); + } + } + + /** + * + */ + private static class DelayRebalanceCommunicationSpi extends TestDelayingCommunicationSpi { + /** {@inheritDoc} */ + @Override protected boolean delayMessage(Message msg, GridIoMessage ioMsg) { + if (msg != null && (msg instanceof GridDhtPartitionDemandMessage || msg instanceof GridDhtPartitionSupplyMessage)) + return true; + + return false; + } + + /** {@inheritDoc} */ + @Override protected int delayMillis() { + return 1_000_000; + } + } } From d3745e9d0a3ff5a64fba494889b7e2605f3af6bb Mon Sep 17 00:00:00 2001 From: Alexey Goncharuk Date: Wed, 7 Feb 2018 21:10:32 +0300 Subject: [PATCH 010/314] IGNITE-7639 Fixed NPE --- .../cluster/DiscoveryDataClusterState.java | 41 +++- .../IgniteClusterActivateDeactivateTest.java | 222 ++++++++---------- 2 files changed, 128 insertions(+), 135 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/DiscoveryDataClusterState.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/DiscoveryDataClusterState.java index dea2ce79c311e..b022754c6670a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/DiscoveryDataClusterState.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/DiscoveryDataClusterState.java @@ -28,22 +28,36 @@ import org.jetbrains.annotations.Nullable; /** - * Discovery data related to cluster state. + * A pojo-object representing current cluster global state. The state includes cluster active flag and cluster + * baseline topology. + *

+ * This object also captures a transitional cluster state, when one or more fields are changing. In this case, + * a {@code transitionReqId} field is set to a non-null value and {@code prevState} captures previous cluster state. + * A joining node catching the cluster in an intermediate state will observe {@code transitionReqId} field to be + * non-null, however the {@code prevState} will not be sent to the joining node. + * + * TODO https://issues.apache.org/jira/browse/IGNITE-7640 This class must be immutable, transitionRes must be set by calling finish(). */ public class DiscoveryDataClusterState implements Serializable { /** */ private static final long serialVersionUID = 0L; - /** */ + /** Flag indicating if the cluster in in active state. */ private final boolean active; - /** */ + /** Current cluster baseline topology. */ @Nullable private final BaselineTopology baselineTopology; - /** */ + /** + * Transition request ID. Set to a non-null value if the cluster is changing it's state. + * The ID is assigned on the initiating node. + */ private final UUID transitionReqId; - /** Topology version for state change exchange. */ + /** + * Topology version in the cluster when state change request was received by the coordinator. + * The exchange fired for the cluster state change will be on version {@code transitionTopVer.nextMinorVersion()}. + */ @GridToStringInclude private final AffinityTopologyVersion transitionTopVer; @@ -51,13 +65,18 @@ public class DiscoveryDataClusterState implements Serializable { @GridToStringExclude private final Set transitionNodes; - /** Local flag for state transition result (global state is updated asynchronously by custom message). */ + /** + * Local flag for state transition active state result (global state is updated asynchronously by custom message), + * {@code null} means that state change is not completed yet. + */ private transient volatile Boolean transitionRes; - /** */ + /** + * Previous cluster state if this state is a transition state and it was not received by a joining node. + */ private transient DiscoveryDataClusterState prevState; - /** */ + /** Transition result error. */ private transient volatile Exception transitionError; /** @@ -86,6 +105,7 @@ static DiscoveryDataClusterState createTransitionState( assert transitionReqId != null; assert transitionTopVer != null; assert !F.isEmpty(transitionNodes) : transitionNodes; + assert prevState != null; return new DiscoveryDataClusterState( prevState, @@ -156,7 +176,7 @@ public boolean transition() { * @return {@code True} if cluster active state change is in progress, {@code false} otherwise. */ public boolean activeStateChanging() { - return transition() && active != prevState.active; + return transition() && (prevState == null || (prevState.active != active)); } /** @@ -202,6 +222,9 @@ public void transitionError(Exception ex) { } /** + * Creates a non-transitional cluster state. This method effectively cleans all fields identifying the + * state as transitional and creates a new state with the state transition result. + * * @param success Transition success status. * @return Cluster state that finished transition. */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java index 71718c9d16cc4..2337329c5ce83 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java @@ -37,7 +37,6 @@ import org.apache.ignite.internal.IgniteClientReconnectAbstractTest; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInternalFuture; -import org.apache.ignite.internal.IgniteKernal; import org.apache.ignite.internal.TestRecordingCommunicationSpi; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsFullMessage; @@ -49,7 +48,6 @@ import org.apache.ignite.lang.IgniteBiPredicate; import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; -import org.apache.ignite.spi.discovery.tcp.TestTcpDiscoverySpi; 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; @@ -84,9 +82,6 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest /** */ private boolean testSpi; - /** */ - private boolean testDiscoSpi; - /** */ private boolean testReconnectSpi; @@ -104,8 +99,6 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest spi.setJoinTimeout(2 * 60_000); } - else if (testDiscoSpi) - cfg.setDiscoverySpi(new TestTcpDiscoverySpi()); ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER); @@ -220,14 +213,14 @@ private void activateSimple(int srvs, int clients, int activateFrom) throws Exce } for (int i = 0; i < srvs + clients; i++) - assertFalse(ignite(i).active()); + assertFalse(ignite(i).cluster().active()); - ignite(activateFrom).active(false); // Should be no-op. + ignite(activateFrom).cluster().active(false); // Should be no-op. - ignite(activateFrom).active(true); + ignite(activateFrom).cluster().active(true); for (int i = 0; i < srvs + clients; i++) - assertTrue(ignite(i).active()); + assertTrue(ignite(i).cluster().active()); for (int i = 0; i < srvs + clients; i++) { for (int c = 0; c < 2; c++) @@ -308,16 +301,14 @@ public void testJoinWhileActivate1_Client() throws Exception { private void joinWhileActivate1(final boolean startClient, final boolean withNewCache) throws Exception { IgniteInternalFuture activeFut = startNodesAndBlockStatusChange(2, 0, 0, false); - IgniteInternalFuture startFut = GridTestUtils.runAsync(new Callable() { - @Override public Void call() throws Exception { - client = startClient; + IgniteInternalFuture startFut = GridTestUtils.runAsync((Callable)() -> { + client = startClient; - ccfgs = withNewCache ? cacheConfigurations2() : cacheConfigurations1(); + ccfgs = withNewCache ? cacheConfigurations2() : cacheConfigurations1(); - startGrid(2); + startGrid(2); - return null; - } + return null; }); TestRecordingCommunicationSpi spi1 = TestRecordingCommunicationSpi.spi(ignite(1)); @@ -376,7 +367,7 @@ private IgniteInternalFuture startNodesAndBlockStatusChange(int srvs, int minorVer = 1; if (initiallyActive && persistenceEnabled()) { - ignite(0).active(true); + ignite(0).cluster().active(true); minorVer++; } @@ -396,11 +387,9 @@ private IgniteInternalFuture startNodesAndBlockStatusChange(int srvs, blockExchangeSingleMessage(spi, STATE_CHANGE_TOP_VER); } - IgniteInternalFuture stateChangeFut = GridTestUtils.runAsync(new Runnable() { - @Override public void run() { - ignite(stateChangeFrom).active(!initiallyActive); - } - }); + IgniteInternalFuture stateChangeFut = GridTestUtils.runAsync(() -> + ignite(stateChangeFrom).cluster().active(!initiallyActive) + ); for (TestRecordingCommunicationSpi spi : spis) spi.waitForBlocked(); @@ -417,17 +406,15 @@ private IgniteInternalFuture startNodesAndBlockStatusChange(int srvs, * @param topVer Exchange topology version. */ private void blockExchangeSingleMessage(TestRecordingCommunicationSpi spi, final AffinityTopologyVersion topVer) { - spi.blockMessages(new IgniteBiPredicate() { - @Override public boolean apply(ClusterNode clusterNode, Message msg) { - if (msg instanceof GridDhtPartitionsSingleMessage) { - GridDhtPartitionsSingleMessage pMsg = (GridDhtPartitionsSingleMessage)msg; - - if (pMsg.exchangeId() != null && pMsg.exchangeId().topologyVersion().equals(topVer)) - return true; - } + spi.blockMessages((IgniteBiPredicate)(clusterNode, msg) -> { + if (msg instanceof GridDhtPartitionsSingleMessage) { + GridDhtPartitionsSingleMessage pMsg = (GridDhtPartitionsSingleMessage)msg; - return false; + if (pMsg.exchangeId() != null && pMsg.exchangeId().topologyVersion().equals(topVer)) + return true; } + + return false; }); } @@ -460,16 +447,14 @@ public void testJoinWhileDeactivate1_Client() throws Exception { private void joinWhileDeactivate1(final boolean startClient, final boolean withNewCache) throws Exception { IgniteInternalFuture activeFut = startNodesAndBlockStatusChange(2, 0, 0, true); - IgniteInternalFuture startFut = GridTestUtils.runAsync(new Callable() { - @Override public Void call() throws Exception { - client = startClient; + IgniteInternalFuture startFut = GridTestUtils.runAsync((Callable)() -> { + client = startClient; - ccfgs = withNewCache ? cacheConfigurations2() : cacheConfigurations1(); + ccfgs = withNewCache ? cacheConfigurations2() : cacheConfigurations1(); - startGrid(2); + startGrid(2); - return null; - } + return null; }); TestRecordingCommunicationSpi spi1 = TestRecordingCommunicationSpi.spi(ignite(1)); @@ -481,7 +466,7 @@ private void joinWhileDeactivate1(final boolean startClient, final boolean withN checkNoCaches(3); - ignite(2).active(true); + ignite(2).cluster().active(true); for (int c = 0; c < 2; c++) checkCache(ignite(2), CACHE_NAME_PREFIX + c, true); @@ -529,30 +514,26 @@ public void testConcurrentJoinAndActivate() throws Exception { final CyclicBarrier b = new CyclicBarrier(START_NODES + 1); - IgniteInternalFuture fut1 = GridTestUtils.runAsync(new Callable() { - @Override public Void call() throws Exception { - b.await(); + IgniteInternalFuture fut1 = GridTestUtils.runAsync(() -> { + b.await(); - Thread.sleep(ThreadLocalRandom.current().nextLong(100) + 1); + U.sleep(ThreadLocalRandom.current().nextLong(100) + 1); - ignite(0).active(true); + ignite(0).cluster().active(true); - return null; - } + return null; }); final AtomicInteger nodeIdx = new AtomicInteger(3); - IgniteInternalFuture fut2 = GridTestUtils.runMultiThreadedAsync(new Callable() { - @Override public Void call() throws Exception { - int idx = nodeIdx.getAndIncrement(); + IgniteInternalFuture fut2 = GridTestUtils.runMultiThreadedAsync((Callable)() -> { + int idx = nodeIdx.getAndIncrement(); - b.await(); + b.await(); - startGrid(idx); + startGrid(idx); - return null; - } + return null; }, START_NODES, "start-node"); fut1.get(); @@ -619,19 +600,19 @@ private void deactivateSimple(int srvs, int clients, int deactivateFrom) throws } if (persistenceEnabled()) - ignite(deactivateFrom).active(true); + ignite(deactivateFrom).cluster().active(true); - ignite(deactivateFrom).active(true); // Should be no-op. + ignite(deactivateFrom).cluster().active(true); // Should be no-op. checkCaches(srvs + clients, CACHES); for (int i = 0; i < srvs + clients; i++) - assertTrue(ignite(i).active()); + assertTrue(ignite(i).cluster().active()); - ignite(deactivateFrom).active(false); + ignite(deactivateFrom).cluster().active(false); for (int i = 0; i < srvs + clients; i++) - assertFalse(ignite(i).active()); + assertFalse(ignite(i).cluster().active()); checkNoCaches(srvs + clients); @@ -648,12 +629,12 @@ private void deactivateSimple(int srvs, int clients, int deactivateFrom) throws checkNoCaches(srvs + clients + 2); for (int i = 0; i < srvs + clients + 2; i++) - assertFalse(ignite(i).active()); + assertFalse(ignite(i).cluster().active()); - ignite(deactivateFrom).active(true); + ignite(deactivateFrom).cluster().active(true); for (int i = 0; i < srvs + clients + 2; i++) { - assertTrue(ignite(i).active()); + assertTrue(ignite(i).cluster().active()); checkCache(ignite(i), CU.UTILITY_CACHE_NAME, true); } @@ -695,7 +676,7 @@ public void testClientReconnectClusterActive() throws Exception { startWithCaches1(SRVS, CLIENTS); if (persistenceEnabled()) - ignite(0).active(true); + ignite(0).cluster().active(true); Ignite srv = ignite(0); Ignite client = ignite(SRVS); @@ -741,7 +722,7 @@ public void testClientReconnectClusterInactive() throws Exception { checkNoCaches(SRVS + CLIENTS); - ignite(0).active(true); + ignite(0).cluster().active(true); checkCache(client, CU.UTILITY_CACHE_NAME, true); @@ -789,39 +770,38 @@ private void clientReconnectClusterDeactivated(final boolean transition) throws IgniteEx client = grid(SRVS); if (persistenceEnabled()) - ignite(0).active(true); + ignite(0).cluster().active(true); checkCache(client, CU.UTILITY_CACHE_NAME, true); checkCaches1(SRVS + CLIENTS); + // Wait for late affinity assignment to finish. + grid(0).context().cache().context().exchange().affinityReadyFuture( + new AffinityTopologyVersion(SRVS + CLIENTS, 1)).get(); + final AffinityTopologyVersion STATE_CHANGE_TOP_VER = new AffinityTopologyVersion(SRVS + CLIENTS + 1, 1); final TestRecordingCommunicationSpi spi1 = transition ? TestRecordingCommunicationSpi.spi(ignite(1)) : null; final AtomicReference stateFut = new AtomicReference<>(); - IgniteClientReconnectAbstractTest.reconnectClientNode(log, client, srv, new Runnable() { - @Override public void run() { - if (transition) { - blockExchangeSingleMessage(spi1, STATE_CHANGE_TOP_VER); - - stateFut.set(GridTestUtils.runAsync(new Runnable() { - @Override public void run() { - srv.active(false); - } - }, "deactivate")); - - try { - U.sleep(500); - } - catch (Exception e) { - e.printStackTrace(); - } + IgniteClientReconnectAbstractTest.reconnectClientNode(log, client, srv, () -> { + if (transition) { + blockExchangeSingleMessage(spi1, STATE_CHANGE_TOP_VER); + + stateFut.set(GridTestUtils.runAsync(() -> srv.cluster().active(false), + "deactivate")); + + try { + U.sleep(500); + } + catch (Exception e) { + e.printStackTrace(); } - else - srv.active(false); } + else + srv.cluster().active(false); }); if (transition) { @@ -839,11 +819,11 @@ private void clientReconnectClusterDeactivated(final boolean transition) throws checkNoCaches(SRVS + CLIENTS); - ignite(0).active(true); + ignite(0).cluster().active(true); checkCache(client, CU.UTILITY_CACHE_NAME, true); - assertTrue(client.active()); + assertTrue(client.cluster().active()); checkCaches1(SRVS + CLIENTS); @@ -900,27 +880,22 @@ private void clientReconnectClusterActivated(final boolean transition) throws Ex final AtomicReference stateFut = new AtomicReference<>(); - IgniteClientReconnectAbstractTest.reconnectClientNode(log, client, srv, new Runnable() { - @Override public void run() { - if (transition) { - blockExchangeSingleMessage(spi1, STATE_CHANGE_TOP_VER); - - stateFut.set(GridTestUtils.runAsync(new Runnable() { - @Override public void run() { - srv.active(true); - } - }, "activate")); - - try { - U.sleep(500); - } - catch (Exception e) { - e.printStackTrace(); - } + IgniteClientReconnectAbstractTest.reconnectClientNode(log, client, srv, () -> { + if (transition) { + blockExchangeSingleMessage(spi1, STATE_CHANGE_TOP_VER); + + stateFut.set(GridTestUtils.runAsync(() -> srv.cluster().active(true), + "activate")); + + try { + U.sleep(500); + } + catch (Exception e) { + e.printStackTrace(); } - else - srv.active(true); } + else + srv.cluster().active(true); }); if (transition) { @@ -989,7 +964,7 @@ public void testInactiveTopologyChanges() throws Exception { checkRecordedMessages(false); - ignite(0).active(true); + ignite(0).cluster().active(true); checkCaches1(SRVS + CLIENTS); @@ -1033,12 +1008,10 @@ private void stateChangeFailover1(boolean activate) throws Exception { client = false; // Start one more node while transition is in progress. - IgniteInternalFuture startFut = GridTestUtils.runAsync(new Callable() { - @Override public Object call() throws Exception { - startGrid(8); + IgniteInternalFuture startFut = GridTestUtils.runAsync(() -> { + startGrid(8); - return null; - } + return null; }, "start-node"); U.sleep(500); @@ -1061,7 +1034,7 @@ private void stateChangeFailover1(boolean activate) throws Exception { if (!activate) { checkNoCaches(9); - ignite(0).active(true); + ignite(0).cluster().active(true); } checkCaches1(9); @@ -1092,19 +1065,16 @@ private void stateChangeFailover2(boolean activate) throws Exception { client = false; // Start more nodes while transition is in progress. - IgniteInternalFuture startFut1 = GridTestUtils.runAsync(new Callable() { - @Override public Object call() throws Exception { - startGrid(8); + IgniteInternalFuture startFut1 = GridTestUtils.runAsync(() -> { + startGrid(8); - return null; - } + return null; }, "start-node1"); - IgniteInternalFuture startFut2 = GridTestUtils.runAsync(new Callable() { - @Override public Object call() throws Exception { - startGrid(9); - return null; - } + IgniteInternalFuture startFut2 = GridTestUtils.runAsync(() -> { + startGrid(9); + + return null; }, "start-node2"); U.sleep(500); @@ -1132,7 +1102,7 @@ private void stateChangeFailover2(boolean activate) throws Exception { if (!activate) { checkNoCaches(10); - ignite(0).active(true); + ignite(0).cluster().active(true); } checkCaches1(10); @@ -1214,7 +1184,7 @@ void checkCache(Ignite node, String cacheName, boolean exp) throws IgniteChecked ((IgniteEx)node).context().state().publicApiActiveState(true); - GridCacheAdapter cache = ((IgniteKernal)node).context().cache().internalCache(cacheName); + GridCacheAdapter cache = ((IgniteEx)node).context().cache().internalCache(cacheName); if (exp) assertNotNull("Cache not found [cache=" + cacheName + ", node=" + node.name() + ']', cache); @@ -1229,7 +1199,7 @@ final void checkNoCaches(int nodes) { for (int i = 0; i < nodes; i++) { grid(i).context().state().publicApiActiveState(true); - GridCacheProcessor cache = ((IgniteKernal)ignite(i)).context().cache(); + GridCacheProcessor cache = ((IgniteEx)ignite(i)).context().cache(); assertTrue(cache.caches().isEmpty()); assertTrue(cache.internalCaches().isEmpty()); From f7c16855ba802d9d47048521aec7e14285e4a281 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Fri, 9 Feb 2018 16:55:15 +0300 Subject: [PATCH 011/314] IGNITE-7540 Prevent page memory metadata corruption during checkpoint and group destroying. - Fixes #3490. Signed-off-by: Alexey Goncharuk --- .../processors/cache/GridCacheProcessor.java | 19 +++ ...tePdsCacheDestroyDuringCheckpointTest.java | 161 ++++++++++++++++++ .../IgnitePdsWithIndexingCoreTestSuite.java | 2 + 3 files changed, 182 insertions(+) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCacheDestroyDuringCheckpointTest.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 1561f25f8e0e4..ec456e17e795d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java @@ -89,6 +89,7 @@ import org.apache.ignite.internal.processors.cache.local.GridLocalCache; import org.apache.ignite.internal.processors.cache.local.atomic.GridLocalAtomicCache; import org.apache.ignite.internal.processors.cache.persistence.DataRegion; +import org.apache.ignite.internal.processors.cache.persistence.DbCheckpointListener; import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager; import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager; import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; @@ -2232,6 +2233,24 @@ public void onExchangeDone( } } + sharedCtx.database().checkpointReadLock(); + + try { + // Do not invoke checkpoint listeners for groups are going to be destroyed to prevent metadata corruption. + for (ExchangeActions.CacheGroupActionData action : exchActions.cacheGroupsToStop()) { + Integer groupId = action.descriptor().groupId(); + CacheGroupContext grp = cacheGrps.get(groupId); + + if (grp != null && grp.persistenceEnabled() && sharedCtx.database() instanceof GridCacheDatabaseSharedManager) { + GridCacheDatabaseSharedManager mngr = (GridCacheDatabaseSharedManager) sharedCtx.database(); + mngr.removeCheckpointListener((DbCheckpointListener) grp.offheap()); + } + } + } + finally { + sharedCtx.database().checkpointReadUnlock(); + } + List> stoppedGroups = new ArrayList<>(); for (ExchangeActions.CacheGroupActionData action : exchActions.cacheGroupsToStop()) { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCacheDestroyDuringCheckpointTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCacheDestroyDuringCheckpointTest.java new file mode 100644 index 0000000000000..72f73aaadb8d5 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCacheDestroyDuringCheckpointTest.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.persistence.db.file; + +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.CacheAtomicityMode; +import org.apache.ignite.cache.CachePeekMode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.DataRegionConfiguration; +import org.apache.ignite.configuration.DataStorageConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; + +/** + * Test for cache creation/deletion with frequent checkpoints. + */ +public class IgnitePdsCacheDestroyDuringCheckpointTest extends GridCommonAbstractTest { + /** */ + private static final String NAME_PREFIX = "CACHE-"; + + /** */ + private static final int NUM_ITERATIONS = 10; + + /** */ + private static final int NUM_CACHES = 10; + + /** */ + private static final int NUM_ENTRIES_PER_CACHE = 200; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setDataStorageConfiguration(createDbConfig()); + + if (getTestIgniteInstanceIndex(gridName) == 1) + cfg.setClientMode(true); + + return cfg; + } + + /** + * @return DB config. + */ + private DataStorageConfiguration createDbConfig() { + DataStorageConfiguration storageCfg = new DataStorageConfiguration(); + storageCfg.setCheckpointFrequency(300); + + DataRegionConfiguration regionConfig = new DataRegionConfiguration(); + regionConfig.setPersistenceEnabled(true); + + storageCfg.setDefaultDataRegionConfiguration(regionConfig); + + return storageCfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + deleteWorkFiles(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + stopAllGrids(); + + deleteWorkFiles(); + } + + /** + * @throws Exception If fail. + */ + public void testCacheCreatePutCheckpointDestroy() throws Exception { + IgniteEx ig = startGrid(0); + ig.active(true); + + for (int j = 0; j < NUM_ITERATIONS; j++) { + Ignite client = startGrid(1); + + for (int i = 0; i < NUM_CACHES; i++) { + IgniteCache cache = ig.cache(NAME_PREFIX + i); + if (cache != null) + cache.destroy(); + } + + populateCache(client); + checkCacheSizes(client); + + client.close(); + } + } + + + /** */ + private void populateCache(Ignite client) { + for (int i = 0; i < NUM_CACHES; i++) { + CacheConfiguration cfg = new CacheConfiguration(); + cfg.setName(NAME_PREFIX + i).setAtomicityMode(CacheAtomicityMode.ATOMIC) + .setBackups(1).setStatisticsEnabled(true).setManagementEnabled(true); + client.getOrCreateCache(cfg); + + IgniteDataStreamer streamer = client.dataStreamer(NAME_PREFIX + i); + + for (int j = 0; j < NUM_ENTRIES_PER_CACHE; j++) { + String bo = i + "|" + j + "|WHATEVER"; + streamer.addData(j, bo); + } + + streamer.close(); + log.info("Streamer closed"); + } + } + + /** */ + private void checkCacheSizes(Ignite client) { + for (int i = 0; i < NUM_CACHES; i++) { + IgniteCache cache = client.getOrCreateCache(NAME_PREFIX + i); + + int size = cache.size(CachePeekMode.ALL); + + if (NUM_ENTRIES_PER_CACHE != size) { + for (Object o : cache) { + log.info("O " + o); + } + assertTrue(false); + } + } + } + + /** + * @throws IgniteCheckedException If fail. + */ + private void deleteWorkFiles() throws IgniteCheckedException { + deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + } +} \ No newline at end of file diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingCoreTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingCoreTestSuite.java index 9f86e0db08107..d3256c2a834da 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingCoreTestSuite.java +++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingCoreTestSuite.java @@ -25,6 +25,7 @@ import org.apache.ignite.internal.processors.cache.persistence.IgnitePersistentStoreCacheGroupsTest; import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsMultiNodePutGetRestartTest; import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsPageEvictionTest; +import org.apache.ignite.internal.processors.cache.persistence.db.file.IgnitePdsCacheDestroyDuringCheckpointTest; import org.apache.ignite.internal.processors.cache.persistence.db.file.IgnitePdsCacheIntegrationTest; import org.apache.ignite.internal.processors.cache.persistence.db.file.IgnitePdsDiskErrorsRecoveringTest; import org.apache.ignite.internal.processors.cache.persistence.db.file.IgnitePdsNoActualWalHistoryTest; @@ -59,6 +60,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(IgniteWalRecoveryPPCTest.class); suite.addTestSuite(IgnitePdsDiskErrorsRecoveringTest.class); + suite.addTestSuite(IgnitePdsCacheDestroyDuringCheckpointTest.class); suite.addTestSuite(IgnitePdsBinaryMetadataOnClusterRestartTest.class); suite.addTestSuite(IgnitePdsMarshallerMappingRestoreOnNodeStartTest.class); From c92f167fc491078f02b9f94fe89edafc2902ebc2 Mon Sep 17 00:00:00 2001 From: ilantukh Date: Wed, 14 Feb 2018 15:40:13 +0300 Subject: [PATCH 012/314] Updated version in properties. --- modules/core/src/main/resources/ignite.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/core/src/main/resources/ignite.properties b/modules/core/src/main/resources/ignite.properties index 1f1f4d05acfc3..a4dffff1056aa 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=2.4.0-SNAPSHOT +ignite.version=2.4.3-SNAPSHOT ignite.build=0 ignite.revision=DEV ignite.rel.date=01011970 From 1ecf348dd429cf7861b414e0e5a7776b72dba281 Mon Sep 17 00:00:00 2001 From: Sergey Chugunov Date: Fri, 16 Feb 2018 16:21:12 +0300 Subject: [PATCH 013/314] IGNITE-7699 BinaryMetadata exchange should not be triggered if metadata was not updated - Fixes #3523. Signed-off-by: Alexey Goncharuk (cherry-picked from commit bcd3881) --- .../processors/cache/binary/BinaryMetadataTransport.java | 8 ++++++-- .../cache/binary/CacheObjectBinaryProcessorImpl.java | 4 ++++ .../IgniteAllBaselineNodesOnlineFullApiSelfTest.java | 4 +--- .../baseline/IgniteBaselineAbstractFullApiSelfTest.java | 2 +- .../IgniteOfflineBaselineNodeFullApiSelfTest.java | 4 +--- .../IgniteOnlineNodeOutOfBaselineFullApiSelfTest.java | 2 +- .../IgniteStableBaselineCachePutAllFailoverTest.java | 3 ++- .../IgniteStableBaselineCacheRemoveFailoverTest.java | 3 ++- 8 files changed, 18 insertions(+), 12 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataTransport.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataTransport.java index def7caa0123e9..247771b9fc487 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataTransport.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataTransport.java @@ -159,7 +159,8 @@ GridFutureAdapter requestMetadataUpdate(BinaryMetadata met MetadataUpdateResultFuture resFut = new MetadataUpdateResultFuture(); if (log.isDebugEnabled()) - log.debug("Requesting metadata update for " + metadata.typeId()); + log.debug("Requesting metadata update for " + metadata.typeId() + "; caller thread is blocked on future " + + resFut); synchronized (this) { unlabeledFutures.add(resFut); @@ -418,6 +419,9 @@ private final class MetadataUpdateAcceptedListener implements CustomEventListene /** {@inheritDoc} */ @Override public void onCustomEvent(AffinityTopologyVersion topVer, ClusterNode snd, MetadataUpdateAcceptedMessage msg) { + if (log.isDebugEnabled()) + log.debug("Received MetadataUpdateAcceptedMessage " + msg); + if (msg.duplicated()) return; @@ -468,7 +472,7 @@ private final class MetadataUpdateAcceptedListener implements CustomEventListene GridFutureAdapter fut = syncMap.get(new SyncKey(typeId, newAcceptedVer)); if (log.isDebugEnabled()) - log.debug("Completing future for " + metaLocCache.get(typeId)); + log.debug("Completing future " + fut + " for " + metaLocCache.get(typeId)); if (fut != null) fut.onDone(MetadataUpdateResult.createSuccessfulResult()); 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 2a374a2728a71..e8328e0b29340 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 @@ -443,6 +443,10 @@ public GridBinaryMarshaller marshaller() { BinaryMetadata mergedMeta = BinaryUtils.mergeMetadata(oldMeta, newMeta0); + //metadata requested to be added is exactly the same as already presented in the cache + if (mergedMeta == oldMeta) + return; + MetadataUpdateResult res = transport.requestMetadataUpdate(mergedMeta).get(); assert res != null; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteAllBaselineNodesOnlineFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteAllBaselineNodesOnlineFullApiSelfTest.java index c6aed7ddc1447..24a3285099706 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteAllBaselineNodesOnlineFullApiSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteAllBaselineNodesOnlineFullApiSelfTest.java @@ -16,10 +16,8 @@ */ package org.apache.ignite.internal.processors.cache.persistence.baseline; -import org.apache.ignite.testframework.GridTestUtils; - /** - * + * Full API cache tests when all online nodes in the cluster are in BaselineTopology. */ public class IgniteAllBaselineNodesOnlineFullApiSelfTest extends IgniteBaselineAbstractFullApiSelfTest { /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteBaselineAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteBaselineAbstractFullApiSelfTest.java index d78c289f2e9e5..182566681171e 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteBaselineAbstractFullApiSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteBaselineAbstractFullApiSelfTest.java @@ -23,7 +23,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheAbstractFullApiSelfTest; /** - * + * Abstract test class for Full API cache tests with presence of BaselineTopology. */ public abstract class IgniteBaselineAbstractFullApiSelfTest extends GridCacheAbstractFullApiSelfTest { /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteOfflineBaselineNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteOfflineBaselineNodeFullApiSelfTest.java index 3fd4974a7ecff..a24185825f64e 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteOfflineBaselineNodeFullApiSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteOfflineBaselineNodeFullApiSelfTest.java @@ -16,10 +16,8 @@ */ package org.apache.ignite.internal.processors.cache.persistence.baseline; -import org.apache.ignite.testframework.GridTestUtils; - /** - * + * Full API cache tests with situation when there is an offline node from BaselineTopology. */ public class IgniteOfflineBaselineNodeFullApiSelfTest extends IgniteBaselineAbstractFullApiSelfTest { /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteOnlineNodeOutOfBaselineFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteOnlineNodeOutOfBaselineFullApiSelfTest.java index 07c475bafdae6..2357de2c0ad51 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteOnlineNodeOutOfBaselineFullApiSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteOnlineNodeOutOfBaselineFullApiSelfTest.java @@ -19,7 +19,7 @@ import org.apache.ignite.testframework.GridTestUtils; /** - * + * Full API cache tests with situation when there is an online node out of BaselineTopology. */ public class IgniteOnlineNodeOutOfBaselineFullApiSelfTest extends IgniteBaselineAbstractFullApiSelfTest { /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteStableBaselineCachePutAllFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteStableBaselineCachePutAllFailoverTest.java index 13f34892ba22d..d92b5bc667460 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteStableBaselineCachePutAllFailoverTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteStableBaselineCachePutAllFailoverTest.java @@ -28,7 +28,8 @@ import org.apache.ignite.testframework.GridTestUtils; /** - * + * Failover cache test with putAll operations executed with presence of BaselineTopology + * when one random node from BLT is constantly restarted during the load. */ public class IgniteStableBaselineCachePutAllFailoverTest extends CachePutAllFailoverAbstractTest { /** */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteStableBaselineCacheRemoveFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteStableBaselineCacheRemoveFailoverTest.java index b3da12e4dfc4a..0654a2f15f348 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteStableBaselineCacheRemoveFailoverTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteStableBaselineCacheRemoveFailoverTest.java @@ -37,7 +37,8 @@ import static org.apache.ignite.cache.CacheMode.PARTITIONED; /** - * + * Failover cache test with remove operations executed with presence of BaselineTopology + * when one random node from BLT is constantly restarted during the load. */ public class IgniteStableBaselineCacheRemoveFailoverTest extends GridCacheAbstractRemoveFailureTest { /** */ From 2458bd08a5b501b3eeb5caf0ae6dcaa2bcccd915 Mon Sep 17 00:00:00 2001 From: EdShangGG Date: Fri, 16 Feb 2018 16:29:49 +0300 Subject: [PATCH 014/314] IGNITE-7676 Add affinity version to snapshot plugin stub - Fixes #3510. Signed-off-by: Alexey Goncharuk (cherry picked from commit b6d21fb) --- .../GridDhtPartitionsExchangeFuture.java | 2 +- .../GridCacheDatabaseSharedManager.java | 64 ++++++++++--------- .../snapshot/IgniteCacheSnapshotManager.java | 9 ++- 3 files changed, 41 insertions(+), 34 deletions(-) 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 695c840de302a..22c96f6d98703 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 @@ -1136,7 +1136,7 @@ private void tryToPerformLocalSnapshotOperation() { try { long start = U.currentTimeMillis(); - IgniteInternalFuture fut = cctx.snapshot().tryStartLocalSnapshotOperation(firstDiscoEvt); + IgniteInternalFuture fut = cctx.snapshot().tryStartLocalSnapshotOperation(firstDiscoEvt, exchId.topologyVersion()); if (fut != null) { fut.get(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java index f1cc4ce562574..66192b9f37bc8 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java @@ -2077,54 +2077,56 @@ private PageMemoryEx getPageMemoryForCacheGroup(int grpId) throws IgniteCheckedE * @param partStates Partition to restore state. */ public void applyUpdatesOnRecovery( - WALIterator it, + @Nullable WALIterator it, IgnitePredicate> recPredicate, IgnitePredicate entryPredicate, Map, T2> partStates ) throws IgniteCheckedException { - while (it.hasNextX()) { - IgniteBiTuple next = it.nextX(); + if (it != null) { + while (it.hasNextX()) { + IgniteBiTuple next = it.nextX(); - WALRecord rec = next.get2(); + WALRecord rec = next.get2(); - if (!recPredicate.apply(next)) - break; + if (!recPredicate.apply(next)) + break; - switch (rec.type()) { - case DATA_RECORD: - checkpointReadLock(); + switch (rec.type()) { + case DATA_RECORD: + checkpointReadLock(); - try { - DataRecord dataRec = (DataRecord) rec; + try { + DataRecord dataRec = (DataRecord)rec; - for (DataEntry dataEntry : dataRec.writeEntries()) { - if (entryPredicate.apply(dataEntry)) { - checkpointReadLock(); + for (DataEntry dataEntry : dataRec.writeEntries()) { + if (entryPredicate.apply(dataEntry)) { + checkpointReadLock(); - try { - int cacheId = dataEntry.cacheId(); + try { + int cacheId = dataEntry.cacheId(); - GridCacheContext cacheCtx = cctx.cacheContext(cacheId); + GridCacheContext cacheCtx = cctx.cacheContext(cacheId); - if (cacheCtx != null) - applyUpdate(cacheCtx, dataEntry); - else if (log != null) - log.warning("Cache (cacheId=" + cacheId + ") is not started, can't apply updates."); - } - finally { - checkpointReadUnlock(); + if (cacheCtx != null) + applyUpdate(cacheCtx, dataEntry); + else if (log != null) + log.warning("Cache (cacheId=" + cacheId + ") is not started, can't apply updates."); + } + finally { + checkpointReadUnlock(); + } } } } - } - finally { - checkpointReadUnlock(); - } + finally { + checkpointReadUnlock(); + } - break; + break; - default: - // Skip other records. + default: + // Skip other records. + } } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteCacheSnapshotManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteCacheSnapshotManager.java index c23addafc9672..16cc8f560c278 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteCacheSnapshotManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteCacheSnapshotManager.java @@ -26,6 +26,7 @@ import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.pagemem.FullPageId; import org.apache.ignite.internal.pagemem.PageMemory; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.CacheGroupContext; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter; @@ -48,9 +49,12 @@ public class IgniteCacheSnapshotManager extends Gri * Try to start local snapshot operation if it's required by discovery event. * * @param discoveryEvent Discovery event. + * @param topVer topology version on the moment when this method was called + * + * @throws IgniteCheckedException if failed */ @Nullable public IgniteInternalFuture tryStartLocalSnapshotOperation( - @Nullable DiscoveryEvent discoveryEvent + @Nullable DiscoveryEvent discoveryEvent, AffinityTopologyVersion topVer ) throws IgniteCheckedException { return null; } @@ -61,7 +65,8 @@ public class IgniteCacheSnapshotManager extends Gri */ @Nullable public IgniteInternalFuture startLocalSnapshotOperation( UUID initiatorNodeId, - T snapshotOperation + T snapshotOperation, + AffinityTopologyVersion topVer ) throws IgniteCheckedException { return null; } From bfdcda7a2a6b5cf64f15ed169d2beb886f131fac Mon Sep 17 00:00:00 2001 From: EdShangGG Date: Mon, 12 Feb 2018 19:36:30 +0300 Subject: [PATCH 015/314] IGNITE-7626 Unify code in test which cleans up persistence directories - Fixes #3477. Signed-off-by: Alexey Goncharuk (cherry picked from commit a0997b9) --- .../JdbcAbstractDmlStatementSelfTest.java | 2 +- .../DummyPersistenceCompatibilityTest.java | 2 +- .../FoldersReuseCompatibilityTest.java | 2 +- ...ngToWalV2SerializerWithCompactionTest.java | 2 +- .../impl/PageMemoryNoLoadSelfTest.java | 2 +- ...ActivateDeactivateTestWithPersistence.java | 4 +-- .../IgniteMarshallerCacheFSRestoreTest.java | 2 +- .../cache/WalModeChangeAbstractSelfTest.java | 4 +-- .../cache/WalModeChangeAdvancedSelfTest.java | 4 +-- .../WalModeChangeCommonAbstractSelfTest.java | 9 ------ .../distributed/Cache64kPartitionsTest.java | 2 +- .../CacheBaselineTopologyTest.java | 4 +-- ...CacheRebalancingPartitionCountersTest.java | 4 +-- ...PageEvictionMultinodeMixedRegionsTest.java | 4 +-- ...aselineAffinityTopologyActivationTest.java | 4 +-- .../IgniteDataStorageMetricsSelfTest.java | 6 ++-- ...PdsBinaryMetadataOnClusterRestartTest.java | 2 +- .../IgnitePdsBinarySortObjectFieldsTest.java | 6 ++-- ...itePdsCacheAssignmentNodeRestartsTest.java | 4 +-- ...IgnitePdsCacheRebalancingAbstractTest.java | 5 ++- .../IgnitePdsContinuousRestartTest.java | 13 ++------ .../IgnitePdsContinuousRestartTest2.java | 14 ++------- .../IgnitePdsDynamicCacheTest.java | 13 ++------ ...IgnitePdsExchangeDuringCheckpointTest.java | 6 ++-- ...rshallerMappingRestoreOnNodeStartTest.java | 5 +-- .../IgnitePdsMultiNodePutGetRestartTest.java | 6 ++-- .../persistence/IgnitePdsPageSizesTest.java | 7 ++--- ...itePdsRecoveryAfterFileCorruptionTest.java | 13 ++------ .../IgnitePdsRemoveDuringRebalancingTest.java | 4 +-- ...itePdsSingleNodePutGetPersistenceTest.java | 7 ++--- .../IgnitePersistentStoreCacheGroupsTest.java | 6 ++-- ...nitePersistentStoreDataStructuresTest.java | 4 +-- ...AllBaselineNodesOnlineFullApiSelfTest.java | 2 +- ...ngBaselineDownCachePutAllFailoverTest.java | 8 ++--- ...ngBaselineDownCacheRemoveFailoverTest.java | 8 ++--- ...gingBaselineUpCachePutAllFailoverTest.java | 8 ++--- ...gingBaselineUpCacheRemoveFailoverTest.java | 8 ++--- ...iteOfflineBaselineNodeFullApiSelfTest.java | 2 +- ...nlineNodeOutOfBaselineFullApiSelfTest.java | 4 +-- ...StableBaselineCachePutAllFailoverTest.java | 4 +-- ...StableBaselineCacheRemoveFailoverTest.java | 4 +-- .../db/IgnitePdsCacheRestoreTest.java | 4 +-- .../db/IgnitePdsDataRegionMetricsTest.java | 4 +-- .../IgnitePdsMultiNodePutGetRestartTest.java | 6 ++-- ...sPageEvictionDuringPartitionClearTest.java | 6 ++-- .../db/IgnitePdsPageEvictionTest.java | 7 ++--- ...PdsRebalancingOnNotStableTopologyTest.java | 6 ++-- .../db/IgnitePdsTransactionsHangTest.java | 5 ++- .../db/IgnitePdsWholeClusterRestartTest.java | 16 ++-------- ...iteCheckpointDirtyPagesForLowLoadTest.java | 4 +-- ...ultPageSizeBackwardsCompatibilityTest.java | 14 ++------- .../file/IgnitePdsCacheIntegrationTest.java | 7 ++--- .../file/IgnitePdsCheckpointSimpleTest.java | 2 +- ...pointSimulationWithRealCpDisabledTest.java | 13 ++------ .../IgnitePdsDiskErrorsRecoveringTest.java | 9 +++--- .../db/file/IgnitePdsEvictionTest.java | 14 ++------- .../file/IgnitePdsNoActualWalHistoryTest.java | 7 ++--- .../file/IgnitePdsThreadInterruptionTest.java | 30 ++++++------------ .../IgniteUidAsConsistentIdMigrationTest.java | 11 ++++--- .../db/wal/IgniteWalFlushFailoverTest.java | 13 ++------ ...lushMultiNodeFailoverAbstractSelfTest.java | 15 ++------- .../wal/IgniteWalHistoryReservationsTest.java | 12 ++----- .../db/wal/IgniteWalRecoveryPPCTest.java | 4 +-- .../IgniteWalRecoverySeveralRestartsTest.java | 6 ++-- .../db/wal/IgniteWalRecoveryTest.java | 4 +-- .../wal/IgniteWalSerializerVersionTest.java | 12 ++----- .../persistence/db/wal/WalCompactionTest.java | 4 +-- .../wal/WalRecoveryTxLogicalRecordsTest.java | 7 ++--- .../db/wal/reader/IgniteWalReaderTest.java | 2 +- .../PagesWriteThrottleSandboxTest.java | 8 ++--- .../pagemem/PagesWriteThrottleSmokeTest.java | 11 +++---- .../AbstractNodeJoinTemplate.java | 6 ++-- .../IgniteChangeGlobalStateAbstractTest.java | 4 +-- .../IgniteStandByClusterTest.java | 6 ++-- .../extended/GridActivateExtensionTest.java | 4 +-- ...iteAbstractStandByClientReconnectTest.java | 5 ++- .../database/IgniteDbAbstractTest.java | 4 +-- .../IgniteDbDynamicCacheSelfTest.java | 4 +-- .../IgniteDbPutGetWithCacheStoreTest.java | 4 +-- .../SwapPathConstructionSelfTest.java | 4 +-- .../ignite/testframework/GridTestUtils.java | 10 ------ .../junits/common/GridCommonAbstractTest.java | 31 +++++-------------- .../ignite/util/GridCommandHandlerTest.java | 11 ++----- ...eckClusterStateBeforeExecuteQueryTest.java | 6 ++-- .../cache/IgniteErrorOnRebalanceTest.java | 2 +- ...NodeWithIndexingPutGetPersistenceTest.java | 7 ++--- .../cache/index/LongIndexNameTest.java | 14 ++------- .../OptimizedMarshallerIndexNameTest.java | 2 +- ...bSingleNodeWithIndexingWalRestoreTest.java | 7 ++--- ...eQueryWithMultipleClassesPerCacheTest.java | 13 ++------ .../IgnitePersistentStoreSchemaLoadTest.java | 13 ++------ ...PartitionOnAffinityRunAtomicCacheTest.java | 8 ++--- ...LockPartitionOnAffinityRunTxCacheTest.java | 8 ++--- ...BaselineCacheQueryNodeRestartSelfTest.java | 5 ++- ...ableBaselineBinObjFieldsQuerySelfTest.java | 9 +++--- ...aselineCacheQueryNodeRestartsSelfTest.java | 4 +-- 96 files changed, 214 insertions(+), 460 deletions(-) diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java index f220b475c3d50..f4c0ca3464ed6 100644 --- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java @@ -150,7 +150,7 @@ protected String getCfgUrl() { private void cleanUpWorkingDir() throws Exception { String workDir = U.defaultWorkDirectory(); - deleteRecursively(U.resolveWorkDirectory(workDir, "marshaller", false)); + U.delete(U.resolveWorkDirectory(workDir, "marshaller", false)); } /** diff --git a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/DummyPersistenceCompatibilityTest.java b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/DummyPersistenceCompatibilityTest.java index b34c921baa810..b36f563337a63 100644 --- a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/DummyPersistenceCompatibilityTest.java +++ b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/DummyPersistenceCompatibilityTest.java @@ -53,7 +53,7 @@ public class DummyPersistenceCompatibilityTest extends IgnitePersistenceCompatib @Override protected void beforeTest() throws Exception { super.beforeTest(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "binary_meta", false)); + U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(), "binary_meta", false)); } /** {@inheritDoc} */ diff --git a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/FoldersReuseCompatibilityTest.java b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/FoldersReuseCompatibilityTest.java index 06b96fc9b7dbf..5a643812add69 100644 --- a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/FoldersReuseCompatibilityTest.java +++ b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/FoldersReuseCompatibilityTest.java @@ -65,7 +65,7 @@ public class FoldersReuseCompatibilityTest extends IgnitePersistenceCompatibilit @Override protected void beforeTest() throws Exception { super.beforeTest(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "binary_meta", false)); + U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(), "binary_meta", false)); } /** {@inheritDoc} */ diff --git a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/MigratingToWalV2SerializerWithCompactionTest.java b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/MigratingToWalV2SerializerWithCompactionTest.java index 0ca3833bf5117..d79790ecd7778 100644 --- a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/MigratingToWalV2SerializerWithCompactionTest.java +++ b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/MigratingToWalV2SerializerWithCompactionTest.java @@ -55,7 +55,7 @@ public class MigratingToWalV2SerializerWithCompactionTest extends IgnitePersiste @Override protected void beforeTest() throws Exception { super.beforeTest(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "binary_meta", false)); + U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(), "binary_meta", false)); } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoLoadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoLoadSelfTest.java index 0f3bf9cd6ebde..6ad977f486a36 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoLoadSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoLoadSelfTest.java @@ -53,7 +53,7 @@ public class PageMemoryNoLoadSelfTest extends GridCommonAbstractTest { /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "pagemem", false)); + U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(), "pagemem", false)); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTestWithPersistence.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTestWithPersistence.java index 76078bcd31f11..58511ee306fdd 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTestWithPersistence.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTestWithPersistence.java @@ -45,14 +45,14 @@ public class IgniteClusterActivateDeactivateTestWithPersistence extends IgniteCl @Override protected void beforeTest() throws Exception { super.beforeTest(); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { super.afterTest(); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMarshallerCacheFSRestoreTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMarshallerCacheFSRestoreTest.java index ac15971f091ca..49f5311bf0e7b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMarshallerCacheFSRestoreTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMarshallerCacheFSRestoreTest.java @@ -110,7 +110,7 @@ private static class SimpleValue { private void cleanUpWorkDir() throws Exception { String workDir = U.defaultWorkDirectory(); - deleteRecursively(U.resolveWorkDirectory(workDir, "marshaller", false)); + U.delete(U.resolveWorkDirectory(workDir, "marshaller", false)); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/WalModeChangeAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/WalModeChangeAbstractSelfTest.java index 048d72003eb97..a4e65aa3e9bfd 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/WalModeChangeAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/WalModeChangeAbstractSelfTest.java @@ -56,7 +56,7 @@ protected WalModeChangeAbstractSelfTest(boolean filterOnCrd, boolean jdbc) { /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { - deleteWorkFiles(); + cleanPersistenceDir(); startGrid(config(SRV_1, false, filterOnCrd)); startGrid(config(SRV_2, false, false)); @@ -71,7 +71,7 @@ protected WalModeChangeAbstractSelfTest(boolean filterOnCrd, boolean jdbc) { @Override protected void afterTestsStopped() throws Exception { stopAllGrids(); - deleteWorkFiles(); + cleanReferences(); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/WalModeChangeAdvancedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/WalModeChangeAdvancedSelfTest.java index 2c6bccff3d7f6..30d55eb678dbd 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/WalModeChangeAdvancedSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/WalModeChangeAdvancedSelfTest.java @@ -47,7 +47,7 @@ public WalModeChangeAdvancedSelfTest() { /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { - deleteWorkFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @@ -56,7 +56,7 @@ public WalModeChangeAdvancedSelfTest() { stopAllGrids(); - deleteWorkFiles(); + cleanPersistenceDir(); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/WalModeChangeCommonAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/WalModeChangeCommonAbstractSelfTest.java index 2e458f28fbb0e..1c41386e39f5b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/WalModeChangeCommonAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/WalModeChangeCommonAbstractSelfTest.java @@ -31,7 +31,6 @@ import org.apache.ignite.internal.util.lang.GridAbsPredicate; import org.apache.ignite.internal.util.lang.IgniteInClosureX; import org.apache.ignite.internal.util.typedef.F; -import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgnitePredicate; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; import org.apache.ignite.testframework.GridTestUtils; @@ -43,7 +42,6 @@ import java.util.concurrent.Callable; import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; /** * Test dynamic WAL mode change. @@ -316,13 +314,6 @@ protected CacheConfiguration cacheConfig(String name, CacheMode mode, CacheAtomi return ccfg; } - /** - * @throws IgniteCheckedException If failed. - */ - protected void deleteWorkFiles() throws IgniteCheckedException { - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); - } - /** * Cache node filter. */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/Cache64kPartitionsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/Cache64kPartitionsTest.java index 7d9df2628be20..95e90c3dbf684 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/Cache64kPartitionsTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/Cache64kPartitionsTest.java @@ -89,6 +89,6 @@ private void checkManyPartitions() throws Exception { /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBaselineTopologyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBaselineTopologyTest.java index 6ccb450319180..bd49da74bd571 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBaselineTopologyTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBaselineTopologyTest.java @@ -81,7 +81,7 @@ public class CacheBaselineTopologyTest extends GridCommonAbstractTest { @Override protected void beforeTest() throws Exception { super.beforeTest(); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @@ -90,7 +90,7 @@ public class CacheBaselineTopologyTest extends GridCommonAbstractTest { stopAllGrids(); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); client = false; } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingPartitionCountersTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingPartitionCountersTest.java index 0676c4503497c..94eb6021c7a29 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingPartitionCountersTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingPartitionCountersTest.java @@ -68,13 +68,13 @@ public class GridCacheRebalancingPartitionCountersTest extends GridCommonAbstrac /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { stopAllGrids(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false)); + U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false)); } /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { stopAllGrids(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false)); + U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false)); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionMultinodeMixedRegionsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionMultinodeMixedRegionsTest.java index 1015e52b220a0..ae6a3fd3e07e4 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionMultinodeMixedRegionsTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionMultinodeMixedRegionsTest.java @@ -43,7 +43,7 @@ public class PageEvictionMultinodeMixedRegionsTest extends PageEvictionMultinode /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); super.beforeTestsStarted(); @@ -54,6 +54,6 @@ public class PageEvictionMultinodeMixedRegionsTest extends PageEvictionMultinode @Override protected void afterTestsStopped() throws Exception { super.afterTestsStopped(); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgniteBaselineAffinityTopologyActivationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgniteBaselineAffinityTopologyActivationTest.java index 1a9393a586807..6b6d666cd021d 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgniteBaselineAffinityTopologyActivationTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgniteBaselineAffinityTopologyActivationTest.java @@ -80,14 +80,14 @@ public class IgniteBaselineAffinityTopologyActivationTest extends GridCommonAbst @Override protected void beforeTest() throws Exception { super.beforeTest(); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { stopAllGrids(false); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgniteDataStorageMetricsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgniteDataStorageMetricsSelfTest.java index 5609995ac8ab6..a752605d276b2 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgniteDataStorageMetricsSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgniteDataStorageMetricsSelfTest.java @@ -57,14 +57,14 @@ public class IgniteDataStorageMetricsSelfTest extends GridCommonAbstractTest { /** {@inheritDoc} */ @Override protected void afterTestsStopped() throws Exception { - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { super.beforeTest(); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @@ -104,7 +104,7 @@ public class IgniteDataStorageMetricsSelfTest extends GridCommonAbstractTest { @Override protected void afterTest() throws Exception { stopAllGrids(); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); super.afterTest(); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsBinaryMetadataOnClusterRestartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsBinaryMetadataOnClusterRestartTest.java index 73c269ada0a69..989626f18ad2c 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsBinaryMetadataOnClusterRestartTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsBinaryMetadataOnClusterRestartTest.java @@ -629,7 +629,7 @@ private void cleanIgniteWorkDir() throws IgniteCheckedException { File baseDirFile = new File(baseDir); for (File f : baseDirFile.listFiles()) - deleteRecursively(U.resolveWorkDirectory(baseDir, f.getName(), false)); + U.delete(U.resolveWorkDirectory(baseDir, f.getName(), false)); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsBinarySortObjectFieldsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsBinarySortObjectFieldsTest.java index 7c8981e5fb43d..70a0203b5ebb2 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsBinarySortObjectFieldsTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsBinarySortObjectFieldsTest.java @@ -86,14 +86,14 @@ public void setVal(Long val) { /** {@inheritDoc} */ @Override protected void afterTestsStopped() throws Exception { - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { super.beforeTest(); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @@ -113,7 +113,7 @@ public void setVal(Long val) { @Override protected void afterTest() throws Exception { stopAllGrids(); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); super.afterTest(); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheAssignmentNodeRestartsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheAssignmentNodeRestartsTest.java index 807f532265d87..0b8f15a430ca3 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheAssignmentNodeRestartsTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheAssignmentNodeRestartsTest.java @@ -80,14 +80,14 @@ public class IgnitePdsCacheAssignmentNodeRestartsTest extends GridCommonAbstract @Override protected void beforeTest() throws Exception { super.beforeTest(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false)); + U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false)); } /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { stopAllGrids(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false)); + U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false)); super.afterTest(); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheRebalancingAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheRebalancingAbstractTest.java index bf20ee48201f8..8139a366ff4b0 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheRebalancingAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheRebalancingAbstractTest.java @@ -54,7 +54,6 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.apache.ignite.transactions.Transaction; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; import static org.apache.ignite.testframework.GridTestUtils.runMultiThreadedAsync; /** @@ -158,14 +157,14 @@ public abstract class IgnitePdsCacheRebalancingAbstractTest extends GridCommonAb @Override protected void beforeTestsStarted() throws Exception { stopAllGrids(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { stopAllGrids(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsContinuousRestartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsContinuousRestartTest.java index fa89bf2afc64a..99614ed7ab697 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsContinuousRestartTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsContinuousRestartTest.java @@ -41,8 +41,6 @@ import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; - /** * */ @@ -104,21 +102,14 @@ public IgnitePdsContinuousRestartTest(boolean cancel) { @Override protected void beforeTestsStarted() throws Exception { stopAllGrids(); - deleteWorkFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { stopAllGrids(); - deleteWorkFiles(); - } - - /** - * @throws IgniteCheckedException If failed. - */ - private void deleteWorkFiles() throws IgniteCheckedException { - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsContinuousRestartTest2.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsContinuousRestartTest2.java index f45fc505860e8..66b20477252c7 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsContinuousRestartTest2.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsContinuousRestartTest2.java @@ -25,7 +25,6 @@ import java.util.concurrent.atomic.AtomicBoolean; 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.CacheAtomicityMode; import org.apache.ignite.cache.CacheWriteSynchronizationMode; @@ -40,8 +39,6 @@ import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; - /** * Cause by https://issues.apache.org/jira/browse/IGNITE-7278 */ @@ -106,21 +103,14 @@ public IgnitePdsContinuousRestartTest2(boolean cancel) { @Override protected void beforeTestsStarted() throws Exception { stopAllGrids(); - deleteWorkFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { stopAllGrids(); - deleteWorkFiles(); - } - - /** - * @throws IgniteCheckedException If failed. - */ - private void deleteWorkFiles() throws IgniteCheckedException { - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsDynamicCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsDynamicCacheTest.java index ba1933f68cca6..871cba3e8a74f 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsDynamicCacheTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsDynamicCacheTest.java @@ -35,8 +35,6 @@ import org.apache.ignite.internal.processors.database.IgniteDbDynamicCacheSelfTest; import org.apache.ignite.internal.util.typedef.internal.U; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; - /** * */ @@ -71,7 +69,7 @@ public class IgnitePdsDynamicCacheTest extends IgniteDbDynamicCacheSelfTest { @Override protected void beforeTestsStarted() throws Exception { super.beforeTestsStarted(); - deleteWorkFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @@ -80,7 +78,7 @@ public class IgnitePdsDynamicCacheTest extends IgniteDbDynamicCacheSelfTest { System.clearProperty(GridCacheDatabaseSharedManager.IGNITE_PDS_CHECKPOINT_TEST_SKIP_SYNC); - deleteWorkFiles(); + cleanPersistenceDir(); } /** @@ -192,13 +190,6 @@ public void testDynamicCacheSavingOnNewNode() throws Exception { assertEquals(i, cache.get(i)); } - /** - * @throws IgniteCheckedException If failed. - */ - private void deleteWorkFiles() throws IgniteCheckedException { - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); - } - /** * */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsExchangeDuringCheckpointTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsExchangeDuringCheckpointTest.java index 2586a1f9476e1..37d55ec3a66ef 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsExchangeDuringCheckpointTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsExchangeDuringCheckpointTest.java @@ -30,8 +30,6 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; - /** * */ @@ -125,12 +123,12 @@ public void testExchangeOnNodeJoin() throws Exception { /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { stopAllGrids(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { stopAllGrids(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsMarshallerMappingRestoreOnNodeStartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsMarshallerMappingRestoreOnNodeStartTest.java index 0429d6b876e6b..27bfe28014cf5 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsMarshallerMappingRestoreOnNodeStartTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsMarshallerMappingRestoreOnNodeStartTest.java @@ -25,6 +25,7 @@ import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; /** @@ -59,8 +60,8 @@ public class IgnitePdsMarshallerMappingRestoreOnNodeStartTest extends GridCommon String tmpDir = System.getProperty("java.io.tmpdir"); - deleteRecursively(Paths.get(tmpDir, "srv0").toFile()); - deleteRecursively(Paths.get(tmpDir, "srv1").toFile()); + U.delete(Paths.get(tmpDir, "srv0").toFile()); + U.delete(Paths.get(tmpDir, "srv1").toFile()); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsMultiNodePutGetRestartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsMultiNodePutGetRestartTest.java index 6a2c9b8a8fe94..3eb17c15015c7 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsMultiNodePutGetRestartTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsMultiNodePutGetRestartTest.java @@ -42,8 +42,6 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; - /** * */ @@ -110,7 +108,7 @@ public IgnitePdsMultiNodePutGetRestartTest() { /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); super.beforeTest(); } @@ -119,7 +117,7 @@ public IgnitePdsMultiNodePutGetRestartTest() { @Override protected void afterTest() throws Exception { super.afterTest(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPageSizesTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPageSizesTest.java index d2ec33a63330f..556f376b70220 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPageSizesTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPageSizesTest.java @@ -28,12 +28,9 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.configuration.WALMode; import org.apache.ignite.internal.IgniteEx; -import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; - /** * */ @@ -66,12 +63,12 @@ public class IgnitePdsPageSizesTest extends GridCommonAbstractTest { /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsRecoveryAfterFileCorruptionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsRecoveryAfterFileCorruptionTest.java index 8e205852bd0b4..8f1009b785150 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsRecoveryAfterFileCorruptionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsRecoveryAfterFileCorruptionTest.java @@ -53,8 +53,6 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; - /** * */ @@ -106,14 +104,14 @@ public class IgnitePdsRecoveryAfterFileCorruptionTest extends GridCommonAbstract @Override protected void beforeTest() throws Exception { stopAllGrids(); - deleteWorkFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { stopAllGrids(); - deleteWorkFiles(); + cleanPersistenceDir(); } /** @@ -394,11 +392,4 @@ private void generateWal( } } } - - /** - * - */ - private void deleteWorkFiles() throws IgniteCheckedException { - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); - } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsRemoveDuringRebalancingTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsRemoveDuringRebalancingTest.java index d2c157b8c70cb..9c96f7fd99153 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsRemoveDuringRebalancingTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsRemoveDuringRebalancingTest.java @@ -84,7 +84,7 @@ public class IgnitePdsRemoveDuringRebalancingTest extends GridCommonAbstractTest @Override protected void beforeTestsStarted() throws Exception { stopAllGrids(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); U.delete(new File(U.getIgniteHome(), DFLT_STORE_DIR)); } @@ -93,7 +93,7 @@ public class IgnitePdsRemoveDuringRebalancingTest extends GridCommonAbstractTest @Override protected void afterTest() throws Exception { G.stopAll(true); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); U.delete(new File(U.getIgniteHome(), DFLT_STORE_DIR)); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsSingleNodePutGetPersistenceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsSingleNodePutGetPersistenceTest.java index 59a6f84035174..53cbb54b3a856 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsSingleNodePutGetPersistenceTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsSingleNodePutGetPersistenceTest.java @@ -18,9 +18,6 @@ package org.apache.ignite.internal.processors.cache.persistence; import org.apache.ignite.internal.processors.database.IgniteDbSingleNodePutGetTest; -import org.apache.ignite.internal.util.typedef.internal.U; - -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; /** * @@ -28,7 +25,7 @@ public class IgnitePdsSingleNodePutGetPersistenceTest extends IgniteDbSingleNodePutGetTest { /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); super.beforeTest(); } @@ -37,6 +34,6 @@ public class IgnitePdsSingleNodePutGetPersistenceTest extends IgniteDbSingleNode @Override protected void afterTest() throws Exception { super.afterTest(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePersistentStoreCacheGroupsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePersistentStoreCacheGroupsTest.java index 1e7ad1b777fe4..ef21bd631d8e7 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePersistentStoreCacheGroupsTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePersistentStoreCacheGroupsTest.java @@ -69,14 +69,14 @@ public class IgnitePersistentStoreCacheGroupsTest extends GridCommonAbstractTest /** {@inheritDoc} */ @Override protected void afterTestsStopped() throws Exception { - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { super.beforeTest(); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @@ -110,7 +110,7 @@ public class IgnitePersistentStoreCacheGroupsTest extends GridCommonAbstractTest @Override protected void afterTest() throws Exception { stopAllGrids(); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); super.afterTest(); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePersistentStoreDataStructuresTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePersistentStoreDataStructuresTest.java index 301c0bc4b7ff8..53e014efaf12e 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePersistentStoreDataStructuresTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePersistentStoreDataStructuresTest.java @@ -63,14 +63,14 @@ public class IgnitePersistentStoreDataStructuresTest extends GridCommonAbstractT /** {@inheritDoc} */ @Override protected void afterTestsStopped() throws Exception { - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { super.beforeTest(); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteAllBaselineNodesOnlineFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteAllBaselineNodesOnlineFullApiSelfTest.java index 24a3285099706..8365f8724b0a0 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteAllBaselineNodesOnlineFullApiSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteAllBaselineNodesOnlineFullApiSelfTest.java @@ -24,7 +24,7 @@ public class IgniteAllBaselineNodesOnlineFullApiSelfTest extends IgniteBaselineA @Override protected void beforeTestsStarted() throws Exception { initStoreStrategy(); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); startGrids(gridCount()); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteChangingBaselineDownCachePutAllFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteChangingBaselineDownCachePutAllFailoverTest.java index 16fa9714a7e24..15384f1bd18a6 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteChangingBaselineDownCachePutAllFailoverTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteChangingBaselineDownCachePutAllFailoverTest.java @@ -60,12 +60,12 @@ public class IgniteChangingBaselineDownCachePutAllFailoverTest extends CachePutA /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); startGrids(GRIDS_COUNT); @@ -78,14 +78,14 @@ public class IgniteChangingBaselineDownCachePutAllFailoverTest extends CachePutA @Override protected void afterTest() throws Exception { stopAllGrids(); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void afterTestsStopped() throws Exception { stopAllGrids(); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteChangingBaselineDownCacheRemoveFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteChangingBaselineDownCacheRemoveFailoverTest.java index 86b58092cceb7..fa96e9cc222b4 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteChangingBaselineDownCacheRemoveFailoverTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteChangingBaselineDownCacheRemoveFailoverTest.java @@ -82,12 +82,12 @@ public class IgniteChangingBaselineDownCacheRemoveFailoverTest extends GridCache /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); startGrids(GRIDS_COUNT); @@ -102,14 +102,14 @@ public class IgniteChangingBaselineDownCacheRemoveFailoverTest extends GridCache @Override protected void afterTestsStopped() throws Exception { stopAllGrids(); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { stopAllGrids(); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteChangingBaselineUpCachePutAllFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteChangingBaselineUpCachePutAllFailoverTest.java index 44c840243117d..219a221166510 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteChangingBaselineUpCachePutAllFailoverTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteChangingBaselineUpCachePutAllFailoverTest.java @@ -59,12 +59,12 @@ public class IgniteChangingBaselineUpCachePutAllFailoverTest extends CachePutAll /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); startGrids(GRIDS_COUNT); @@ -77,14 +77,14 @@ public class IgniteChangingBaselineUpCachePutAllFailoverTest extends CachePutAll @Override protected void afterTest() throws Exception { stopAllGrids(); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void afterTestsStopped() throws Exception { stopAllGrids(); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteChangingBaselineUpCacheRemoveFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteChangingBaselineUpCacheRemoveFailoverTest.java index 8391e0cc74b43..b2642e96b82c1 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteChangingBaselineUpCacheRemoveFailoverTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteChangingBaselineUpCacheRemoveFailoverTest.java @@ -81,12 +81,12 @@ public class IgniteChangingBaselineUpCacheRemoveFailoverTest extends GridCacheAb /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); startGrids(GRIDS_COUNT); @@ -99,14 +99,14 @@ public class IgniteChangingBaselineUpCacheRemoveFailoverTest extends GridCacheAb @Override protected void afterTestsStopped() throws Exception { stopAllGrids(); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { stopAllGrids(); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteOfflineBaselineNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteOfflineBaselineNodeFullApiSelfTest.java index a24185825f64e..2282165d7164f 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteOfflineBaselineNodeFullApiSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteOfflineBaselineNodeFullApiSelfTest.java @@ -24,7 +24,7 @@ public class IgniteOfflineBaselineNodeFullApiSelfTest extends IgniteBaselineAbst @Override protected void beforeTestsStarted() throws Exception { initStoreStrategy(); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); startGrids(gridCount()); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteOnlineNodeOutOfBaselineFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteOnlineNodeOutOfBaselineFullApiSelfTest.java index 2357de2c0ad51..3bd8c0fd51f8c 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteOnlineNodeOutOfBaselineFullApiSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteOnlineNodeOutOfBaselineFullApiSelfTest.java @@ -16,8 +16,6 @@ */ package org.apache.ignite.internal.processors.cache.persistence.baseline; -import org.apache.ignite.testframework.GridTestUtils; - /** * Full API cache tests with situation when there is an online node out of BaselineTopology. */ @@ -26,7 +24,7 @@ public class IgniteOnlineNodeOutOfBaselineFullApiSelfTest extends IgniteBaseline @Override protected void beforeTestsStarted() throws Exception { initStoreStrategy(); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); startGrids(gridCount()); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteStableBaselineCachePutAllFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteStableBaselineCachePutAllFailoverTest.java index d92b5bc667460..80f3b16d9bfe8 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteStableBaselineCachePutAllFailoverTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteStableBaselineCachePutAllFailoverTest.java @@ -85,7 +85,7 @@ public class IgniteStableBaselineCachePutAllFailoverTest extends CachePutAllFail /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); startGrids(GRIDS_COUNT); @@ -100,6 +100,6 @@ public class IgniteStableBaselineCachePutAllFailoverTest extends CachePutAllFail @Override protected void afterTestsStopped() throws Exception { stopAllGrids(); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteStableBaselineCacheRemoveFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteStableBaselineCacheRemoveFailoverTest.java index 0654a2f15f348..e71f33fa0e0da 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteStableBaselineCacheRemoveFailoverTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteStableBaselineCacheRemoveFailoverTest.java @@ -81,7 +81,7 @@ public class IgniteStableBaselineCacheRemoveFailoverTest extends GridCacheAbstra /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); startGrids(GRIDS_COUNT); @@ -96,7 +96,7 @@ public class IgniteStableBaselineCacheRemoveFailoverTest extends GridCacheAbstra @Override protected void afterTestsStopped() throws Exception { stopAllGrids(); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsCacheRestoreTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsCacheRestoreTest.java index 7da765b61b2dc..525f36a7d18ed 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsCacheRestoreTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsCacheRestoreTest.java @@ -78,14 +78,14 @@ public class IgnitePdsCacheRestoreTest extends GridCommonAbstractTest { @Override protected void beforeTest() throws Exception { super.beforeTest(); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { stopAllGrids(); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); super.afterTest(); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsDataRegionMetricsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsDataRegionMetricsTest.java index af65d1d614837..4b7d6aeb8b7c1 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsDataRegionMetricsTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsDataRegionMetricsTest.java @@ -94,14 +94,14 @@ public class IgnitePdsDataRegionMetricsTest extends GridCommonAbstractTest { @Override protected void beforeTest() throws Exception { super.beforeTest(); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { stopAllGrids(); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); super.afterTest(); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsMultiNodePutGetRestartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsMultiNodePutGetRestartTest.java index 615e1083c7dbb..690c9a652809a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsMultiNodePutGetRestartTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsMultiNodePutGetRestartTest.java @@ -42,8 +42,6 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; - /** * */ @@ -95,7 +93,7 @@ public class IgnitePdsMultiNodePutGetRestartTest extends GridCommonAbstractTest /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); super.beforeTest(); } @@ -104,7 +102,7 @@ public class IgnitePdsMultiNodePutGetRestartTest extends GridCommonAbstractTest @Override protected void afterTest() throws Exception { super.afterTest(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsPageEvictionDuringPartitionClearTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsPageEvictionDuringPartitionClearTest.java index 3dfdc57393cb9..8858206c12baa 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsPageEvictionDuringPartitionClearTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsPageEvictionDuringPartitionClearTest.java @@ -36,8 +36,6 @@ import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; - /** * */ @@ -89,7 +87,7 @@ public class IgnitePdsPageEvictionDuringPartitionClearTest extends GridCommonAbs */ public void testPageEvictionOnNodeStart() throws Exception { for (int r = 0; r < 3; r++) { - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); startGrids(2); @@ -138,7 +136,7 @@ public void testPageEvictionOnNodeStart() throws Exception { finally { stopAllGrids(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); } } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsPageEvictionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsPageEvictionTest.java index 47d0cb2c869da..931e0d6ec9296 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsPageEvictionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsPageEvictionTest.java @@ -33,14 +33,11 @@ import org.apache.ignite.configuration.WALMode; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.util.typedef.internal.S; -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 static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; - /** * */ @@ -88,7 +85,7 @@ public class IgnitePdsPageEvictionTest extends GridCommonAbstractTest { /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); stopAllGrids(); @@ -99,7 +96,7 @@ public class IgnitePdsPageEvictionTest extends GridCommonAbstractTest { @Override protected void afterTest() throws Exception { stopAllGrids(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsRebalancingOnNotStableTopologyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsRebalancingOnNotStableTopologyTest.java index 893ecb5b317c1..96901749cd4a9 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsRebalancingOnNotStableTopologyTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsRebalancingOnNotStableTopologyTest.java @@ -36,8 +36,6 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.apache.ignite.testframework.junits.multijvm.IgniteProcessProxy; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; - /** * We start writing to unstable cluster. * After that we start killing node. @@ -192,14 +190,14 @@ public void test() throws Exception { @Override protected void beforeTest() throws Exception { stopAllGrids(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { stopAllGrids(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsTransactionsHangTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsTransactionsHangTest.java index 22d16657ee9f8..f9d30e2624ef3 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsTransactionsHangTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsTransactionsHangTest.java @@ -47,7 +47,6 @@ import org.apache.ignite.transactions.TransactionIsolation; import org.jsr166.LongAdder8; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ; @@ -89,12 +88,12 @@ public class IgnitePdsTransactionsHangTest extends GridCommonAbstractTest { /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsWholeClusterRestartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsWholeClusterRestartTest.java index f52f4467557a1..dc6f0ec772c87 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsWholeClusterRestartTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsWholeClusterRestartTest.java @@ -32,12 +32,9 @@ import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.configuration.WALMode; -import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.spi.checkpoint.noop.NoopCheckpointSpi; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; - /** * */ @@ -87,24 +84,17 @@ public class IgnitePdsWholeClusterRestartTest extends GridCommonAbstractTest { @Override protected void beforeTestsStarted() throws Exception { stopAllGrids(); - deleteWorkFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { stopAllGrids(); - deleteWorkFiles(); - } - - /** - * @throws IgniteCheckedException If failed. - */ - private void deleteWorkFiles() throws IgniteCheckedException { - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); } - /** + /** * @throws Exception if failed. */ public void testRestarts() throws Exception { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/checkpoint/IgniteCheckpointDirtyPagesForLowLoadTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/checkpoint/IgniteCheckpointDirtyPagesForLowLoadTest.java index 782949ffffe14..0478e6ee815e5 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/checkpoint/IgniteCheckpointDirtyPagesForLowLoadTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/checkpoint/IgniteCheckpointDirtyPagesForLowLoadTest.java @@ -83,8 +83,8 @@ public class IgniteCheckpointDirtyPagesForLowLoadTest extends GridCommonAbstract @Override protected void beforeTest() throws Exception { stopAllGrids(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false)); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "temp", false)); + cleanPersistenceDir(); + U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(), "temp", false)); } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/DefaultPageSizeBackwardsCompatibilityTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/DefaultPageSizeBackwardsCompatibilityTest.java index 9e01f7becb376..e0e277c4e35bb 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/DefaultPageSizeBackwardsCompatibilityTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/DefaultPageSizeBackwardsCompatibilityTest.java @@ -26,14 +26,11 @@ import org.apache.ignite.configuration.DataRegionConfiguration; import org.apache.ignite.configuration.DataStorageConfiguration; 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; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; - /** * */ @@ -90,14 +87,14 @@ public class DefaultPageSizeBackwardsCompatibilityTest extends GridCommonAbstrac @Override protected void beforeTest() throws Exception { stopAllGrids(); - deleteWorkFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { stopAllGrids(); - deleteWorkFiles(); + cleanPersistenceDir(); } /** @@ -136,11 +133,4 @@ public void testStartFrom2kDefaultStore() throws Exception { for (int i = 0; i < ENTRIES_COUNT; i++) assertEquals((Integer)i, cache.get(i)); } - - /** - * @throws IgniteCheckedException If failed. - */ - private void deleteWorkFiles() throws IgniteCheckedException { - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); - } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCacheIntegrationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCacheIntegrationTest.java index 7d51b461ea5af..1ba723dddde4b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCacheIntegrationTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCacheIntegrationTest.java @@ -38,15 +38,12 @@ import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.processors.cache.GridCacheAdapter; import org.apache.ignite.internal.util.typedef.internal.S; -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.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; - /** * */ @@ -101,14 +98,14 @@ public class IgnitePdsCacheIntegrationTest extends GridCommonAbstractTest { /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { super.afterTest(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCheckpointSimpleTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCheckpointSimpleTest.java index 4655fccf376f7..0cfcec8450025 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCheckpointSimpleTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCheckpointSimpleTest.java @@ -51,7 +51,7 @@ public class IgnitePdsCheckpointSimpleTest extends GridCommonAbstractTest { @Override protected void beforeTestsStarted() throws Exception { super.beforeTestsStarted(); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCheckpointSimulationWithRealCpDisabledTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCheckpointSimulationWithRealCpDisabledTest.java index 6f222f28fb42b..708d2b5aa20b4 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCheckpointSimulationWithRealCpDisabledTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCheckpointSimulationWithRealCpDisabledTest.java @@ -81,8 +81,6 @@ import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; - /** * Test simulated chekpoints, * Disables integrated check pointer thread @@ -131,14 +129,14 @@ public class IgnitePdsCheckpointSimulationWithRealCpDisabledTest extends GridCom @Override protected void beforeTest() throws Exception { stopAllGrids(); - deleteWorkFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { stopAllGrids(); - deleteWorkFiles(); + cleanPersistenceDir(); } /** @@ -996,13 +994,6 @@ private IgniteBiTuple, WALPointer> runCheckpointing( return F.t((Map)resMap, start); } - /** - * - */ - private void deleteWorkFiles() throws IgniteCheckedException { - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); - } - /** * Initializes page. * @param mem page memory implementation. diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsDiskErrorsRecoveringTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsDiskErrorsRecoveringTest.java index 35115515ddd07..d396f52412e5a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsDiskErrorsRecoveringTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsDiskErrorsRecoveringTest.java @@ -45,13 +45,11 @@ import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory; import org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager; import org.apache.ignite.internal.util.typedef.G; -import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.junit.Assert; import static org.apache.ignite.IgniteSystemProperties.IGNITE_WAL_MMAP; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; /** * Tests node recovering after disk errors during interaction with persistent storage. @@ -82,14 +80,14 @@ public class IgnitePdsDiskErrorsRecoveringTest extends GridCommonAbstractTest { @Override protected void beforeTest() throws Exception { stopAllGrids(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { stopAllGrids(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); failPageStoreDiskOperations = false; diskSpaceBytes = DFLT_DISK_SPACE_BYTES; @@ -344,6 +342,9 @@ public LimitedSizeFileIO(FileIO delegate, AtomicLong availableSpaceBytes) { } } + /** + * + */ private static class LimitedSizeFileIOFactory implements FileIOFactory { /** Serial version uid. */ private static final long serialVersionUID = 0L; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsEvictionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsEvictionTest.java index 117be9bb189a1..9dd01463bb22a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsEvictionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsEvictionTest.java @@ -38,12 +38,9 @@ import org.apache.ignite.internal.processors.cache.persistence.DummyPageIO; import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO; import org.apache.ignite.internal.util.typedef.internal.CU; -import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; - /** * Test for page evictions. */ @@ -102,7 +99,7 @@ private DataStorageConfiguration createDbConfig() { @Override protected void beforeTestsStarted() throws Exception { super.beforeTestsStarted(); - deleteWorkFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @@ -111,7 +108,7 @@ private DataStorageConfiguration createDbConfig() { stopAllGrids(); - deleteWorkFiles(); + cleanPersistenceDir(); } /** @@ -321,11 +318,4 @@ private PageMemory getMemory(IgniteEx ig) throws Exception { return db.dataRegion(null).pageMemory(); } - - /** - * @throws IgniteCheckedException If fail. - */ - private void deleteWorkFiles() throws IgniteCheckedException { - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); - } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsNoActualWalHistoryTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsNoActualWalHistoryTest.java index 61f92c5796aab..48372ef4aad95 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsNoActualWalHistoryTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsNoActualWalHistoryTest.java @@ -35,11 +35,8 @@ import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager; 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.testframework.junits.common.GridCommonAbstractTest; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; - /** * */ @@ -88,14 +85,14 @@ public class IgnitePdsNoActualWalHistoryTest extends GridCommonAbstractTest { @Override protected void beforeTest() throws Exception { stopAllGrids(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { stopAllGrids(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsThreadInterruptionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsThreadInterruptionTest.java index 6cd3c1f77e77b..0f33d55e0d4ef 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsThreadInterruptionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsThreadInterruptionTest.java @@ -20,7 +20,6 @@ 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.cache.affinity.rendezvous.RendezvousAffinityFunction; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.DataPageEvictionMode; @@ -29,7 +28,6 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.configuration.WALMode; import org.apache.ignite.internal.processors.cache.persistence.file.AsyncFileIOFactory; -import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.jsr166.ThreadLocalRandom8; @@ -93,7 +91,7 @@ private DataStorageConfiguration storageConfiguration() { @Override protected void beforeTest() throws Exception { super.beforeTestsStarted(); - deleteWorkFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @@ -102,7 +100,7 @@ private DataStorageConfiguration storageConfiguration() { stopAllGrids(); - deleteWorkFiles(); + cleanPersistenceDir(); } /** @@ -125,26 +123,25 @@ public void testInterruptsOnLFSRead() throws Exception { final IgniteCache cache = ignite.cache(CACHE_NAME); - for (int i=0; i < maxKey; i++) { + + for (int i=0; i < maxKey; i++) cache.put(i, payload); - } final AtomicReference fail = new AtomicReference<>(); Runnable clo = new Runnable() { - @Override - public void run() { - cache.get(ThreadLocalRandom8.current().nextInt(maxKey / 5)); + @Override public void run() { + cache.get(ThreadLocalRandom.current().nextInt(maxKey / 5)); } }; + for (int i = 0; i < workers.length; i++) { workers[i] = new Thread(clo); workers[i].setName("reader-" + i); workers[i].setUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() { @Override public void uncaughtException(Thread t, Throwable e) { fail.compareAndSet(null, e); - } }); } @@ -167,9 +164,7 @@ public void run() { Throwable t = fail.get(); - assert t == null : t; - - + assertNull(t); int verifiedKeys = 0; @@ -244,7 +239,7 @@ public void testInterruptsOnWALWrite() throws Exception { Throwable t = fail.get(); - assert t == null : t; + assertNull(t); IgniteCache cache = ignite.cache(CACHE_NAME); @@ -263,11 +258,4 @@ public void testInterruptsOnWALWrite() throws Exception { log.info("Verified keys: " + verifiedKeys); } - - /** - * @throws IgniteCheckedException If fail. - */ - private void deleteWorkFiles() throws IgniteCheckedException { - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false)); - } } \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/filename/IgniteUidAsConsistentIdMigrationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/filename/IgniteUidAsConsistentIdMigrationTest.java index 1f322decc24f2..419ad63da87a5 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/filename/IgniteUidAsConsistentIdMigrationTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/filename/IgniteUidAsConsistentIdMigrationTest.java @@ -30,6 +30,7 @@ import org.apache.ignite.configuration.DataRegionConfiguration; import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; import org.apache.ignite.internal.processors.cache.persistence.filename.PdsConsistentIdProcessor; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.testframework.GridStringLogger; @@ -105,17 +106,17 @@ private void deleteWorkFiles() throws IgniteCheckedException { boolean ok = true; if (pstStoreCustomPath != null) - ok &= deleteRecursively(pstStoreCustomPath); + ok &= U.delete(pstStoreCustomPath); else - ok &= deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false)); + ok &= U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(), FilePageStoreManager.DFLT_STORE_DIR, false)); if (pstWalArchCustomPath != null) - ok &= deleteRecursively(pstWalArchCustomPath); + ok &= U.delete(pstWalArchCustomPath); if (pstWalStoreCustomPath != null) - ok &= deleteRecursively(pstWalStoreCustomPath); + ok &= U.delete(pstWalStoreCustomPath); - ok &= deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "binary_meta", false)); + ok &= U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(), "binary_meta", false)); if (failIfDeleteNotCompleted) assertTrue(ok); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFailoverTest.java index 386b83ca44cf2..946b4e80c52d5 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFailoverTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFailoverTest.java @@ -39,7 +39,6 @@ import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory; import org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager; import org.apache.ignite.internal.util.lang.GridAbsPredicate; -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; @@ -49,7 +48,6 @@ import static java.nio.file.StandardOpenOption.CREATE; import static java.nio.file.StandardOpenOption.READ; import static java.nio.file.StandardOpenOption.WRITE; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; /** * @@ -66,14 +64,14 @@ public class IgniteWalFlushFailoverTest extends GridCommonAbstractTest { /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { - deleteWorkFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { stopAllGrids(); - deleteWorkFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @@ -169,13 +167,6 @@ private void flushingErrorTest() throws Exception { }, getTestTimeout()); } - /** - * @throws IgniteCheckedException If failed. - */ - private void deleteWorkFiles() throws IgniteCheckedException { - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); - } - /** * Create File I/O which fails after second attempt to write to File */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushMultiNodeFailoverAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushMultiNodeFailoverAbstractSelfTest.java index 9b110df0f62d0..1259c3c489ccc 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushMultiNodeFailoverAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushMultiNodeFailoverAbstractSelfTest.java @@ -39,7 +39,6 @@ import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory; import org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager; import org.apache.ignite.internal.util.lang.GridAbsPredicate; -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; @@ -52,7 +51,6 @@ import static java.nio.file.StandardOpenOption.CREATE; import static java.nio.file.StandardOpenOption.READ; import static java.nio.file.StandardOpenOption.WRITE; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; /** * Tests error recovery while node flushing @@ -76,14 +74,14 @@ public abstract class IgniteWalFlushMultiNodeFailoverAbstractSelfTest extends Gr @Override protected void beforeTest() throws Exception { stopAllGrids(); - deleteWorkFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { stopAllGrids(); - deleteWorkFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @@ -138,7 +136,6 @@ public void testFailAfterStart() throws Exception { * @throws Exception if failed. */ public void failWhilePut(boolean failWhileStart) throws Exception { - final Ignite grid = startGridsMultiThreaded(gridCount()); IgniteWriteAheadLogManager wal = ((IgniteKernal)grid).context().cache().context().wal(); @@ -215,14 +212,6 @@ public void failWhilePut(boolean failWhileStart) throws Exception { assertEquals(cache.get(i), "testValue" + i); } - - /** - * @throws IgniteCheckedException - */ - private void deleteWorkFiles() throws IgniteCheckedException { - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); - } - /** * Create File I/O which fails after second attempt to write to File */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalHistoryReservationsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalHistoryReservationsTest.java index 66a8aa9c2a366..8bf35b39b7980 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalHistoryReservationsTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalHistoryReservationsTest.java @@ -43,7 +43,6 @@ import org.junit.Assert; import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_WAL_REBALANCE_THRESHOLD; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; /** * @@ -86,7 +85,7 @@ public class IgniteWalHistoryReservationsTest extends GridCommonAbstractTest { @Override protected void beforeTestsStarted() throws Exception { stopAllGrids(); - deleteWorkFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @@ -97,7 +96,7 @@ public class IgniteWalHistoryReservationsTest extends GridCommonAbstractTest { stopAllGrids(); - deleteWorkFiles(); + cleanPersistenceDir(); } /** @@ -450,13 +449,6 @@ public void testNodeLeftDuringExchange() throws Exception { awaitPartitionMapExchange(); } - /** - * @throws IgniteCheckedException If failed. - */ - private void deleteWorkFiles() throws IgniteCheckedException { - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); - } - /** * */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryPPCTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryPPCTest.java index f3c2c99f4854e..c67155b23f559 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryPPCTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryPPCTest.java @@ -125,14 +125,14 @@ public class IgniteWalRecoveryPPCTest extends GridCommonAbstractTest { @Override protected void beforeTest() throws Exception { stopAllGrids(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false)); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { stopAllGrids(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false)); + cleanPersistenceDir(); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoverySeveralRestartsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoverySeveralRestartsTest.java index 699fe81661665..20862704a232f 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoverySeveralRestartsTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoverySeveralRestartsTest.java @@ -38,8 +38,6 @@ import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; - /** * */ @@ -100,7 +98,7 @@ public class IgniteWalRecoverySeveralRestartsTest extends GridCommonAbstractTest /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); super.beforeTest(); } @@ -109,7 +107,7 @@ public class IgniteWalRecoverySeveralRestartsTest extends GridCommonAbstractTest @Override protected void afterTest() throws Exception { super.afterTest(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java index 7bb550d2e3946..dbb91ee5e5d3a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java @@ -179,7 +179,7 @@ public class IgniteWalRecoveryTest extends GridCommonAbstractTest { @Override protected void beforeTest() throws Exception { stopAllGrids(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); cacheName = "partitioned"; } @@ -190,7 +190,7 @@ public class IgniteWalRecoveryTest extends GridCommonAbstractTest { logOnly = false; - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalSerializerVersionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalSerializerVersionTest.java index fc61632af017e..3959e6698c681 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalSerializerVersionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalSerializerVersionTest.java @@ -50,7 +50,6 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import static org.apache.ignite.IgniteSystemProperties.IGNITE_WAL_SERIALIZER_VERSION; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; import static org.apache.ignite.transactions.TransactionState.PREPARED; /** @@ -288,7 +287,7 @@ private void check(Checker checker) throws Exception { stopAllGrids(); - deleteWorkFiles(); + cleanPersistenceDir(); System.clearProperty(IGNITE_WAL_SERIALIZER_VERSION); } @@ -299,7 +298,7 @@ private void check(Checker checker) throws Exception { stopAllGrids(); - deleteWorkFiles(); + cleanPersistenceDir(); System.clearProperty(IGNITE_WAL_SERIALIZER_VERSION); } @@ -309,13 +308,6 @@ private void check(Checker checker) throws Exception { System.clearProperty(IGNITE_WAL_SERIALIZER_VERSION); } - /** - * @throws IgniteCheckedException If failed. - */ - private void deleteWorkFiles() throws IgniteCheckedException { - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); - } - /** * */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalCompactionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalCompactionTest.java index d20434769e8d9..6b79d906a4937 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalCompactionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalCompactionTest.java @@ -90,14 +90,14 @@ public class WalCompactionTest extends GridCommonAbstractTest { @Override protected void beforeTest() throws Exception { stopAllGrids(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false)); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { stopAllGrids(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false)); + cleanPersistenceDir(); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalRecoveryTxLogicalRecordsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalRecoveryTxLogicalRecordsTest.java index 0b3d0166a9242..287f8d007da7c 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalRecoveryTxLogicalRecordsTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalRecoveryTxLogicalRecordsTest.java @@ -59,14 +59,11 @@ 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.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.apache.ignite.transactions.Transaction; import org.junit.Assert; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; - /** * */ @@ -136,12 +133,12 @@ public class WalRecoveryTxLogicalRecordsTest extends GridCommonAbstractTest { /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java index 63c219be7075f..3bd1582b02c42 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java @@ -169,7 +169,7 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest { * @throws IgniteCheckedException If failed. */ private void deleteWorkFiles() throws Exception { - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSandboxTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSandboxTest.java index 9529f5965c681..c417b07995ebb 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSandboxTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSandboxTest.java @@ -44,8 +44,6 @@ import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; - /** * Test to visualize and debug {@link PagesWriteThrottle}. * Prints puts/gets rate, number of dirty pages, pages written in current checkpoint and pages in checkpoint buffer. @@ -253,8 +251,8 @@ private TestValue(int v1, int v2) { /** * @throws IgniteCheckedException If failed. */ - private void deleteWorkFiles() throws IgniteCheckedException { - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "snapshot", false)); + private void deleteWorkFiles() throws Exception { + cleanPersistenceDir(); + U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(), "snapshot", false)); } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSmokeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSmokeTest.java index 57fecbe66844b..9f1342fd9d3b7 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSmokeTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSmokeTest.java @@ -27,7 +27,6 @@ import java.util.concurrent.locks.LockSupport; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; -import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheWriteSynchronizationMode; import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; @@ -54,7 +53,6 @@ import static java.nio.file.StandardOpenOption.CREATE; import static java.nio.file.StandardOpenOption.READ; import static java.nio.file.StandardOpenOption.WRITE; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; /** * @@ -271,11 +269,12 @@ private TestValue(int v1, int v2) { } /** - * @throws IgniteCheckedException If failed. + * @throws Exception If failed. */ - private void deleteWorkFiles() throws IgniteCheckedException { - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "snapshot", false)); + private void deleteWorkFiles() throws Exception { + cleanPersistenceDir(); + + U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(), "snapshot", false)); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/AbstractNodeJoinTemplate.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/AbstractNodeJoinTemplate.java index 66e5b172b1612..0dee9163dda93 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/AbstractNodeJoinTemplate.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/AbstractNodeJoinTemplate.java @@ -32,7 +32,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheAdapter; import org.apache.ignite.internal.util.typedef.CI1; import org.apache.ignite.internal.util.typedef.internal.CU; -import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteCallable; import org.apache.ignite.lang.IgniteClosure; import org.apache.ignite.lang.IgniteInClosure; @@ -42,7 +41,6 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.junit.Assert; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; import static org.apache.ignite.internal.util.IgniteUtils.field; /** @@ -214,7 +212,7 @@ protected static Map caches(IgniteEx ig){ stopAllGrids(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); } /** {@inheritDoc} */ @@ -223,7 +221,7 @@ protected static Map caches(IgniteEx ig){ stopAllGrids(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateAbstractTest.java index 71d23f2f5fcaf..c30c8e90f65fd 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateAbstractTest.java @@ -61,7 +61,7 @@ public abstract class IgniteChangeGlobalStateAbstractTest extends GridCommonAbst nodes.clear(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), testName(), true)); + U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(), testName(), true)); startPrimaryNodes(primaryNodes()); @@ -84,7 +84,7 @@ public abstract class IgniteChangeGlobalStateAbstractTest extends GridCommonAbst nodes.clear(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), testName(), true)); + U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(), testName(), true)); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteStandByClusterTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteStandByClusterTest.java index 300f9f8e963b0..c62565fc572be 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteStandByClusterTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteStandByClusterTest.java @@ -53,8 +53,6 @@ import org.jetbrains.annotations.Nullable; import org.junit.Assert; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; - /** * */ @@ -499,7 +497,7 @@ public static class StanByClusterTestProvider implements PluginProvider, IgniteP stopAllGrids(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, true)); + cleanPersistenceDir(); } /** @@ -510,6 +508,6 @@ public static class StanByClusterTestProvider implements PluginProvider, IgniteP stopAllGrids(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, true)); + cleanPersistenceDir(); } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/extended/GridActivateExtensionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/extended/GridActivateExtensionTest.java index 6ca29d84623b2..20031553dde86 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/extended/GridActivateExtensionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/extended/GridActivateExtensionTest.java @@ -82,7 +82,7 @@ public class GridActivateExtensionTest extends GridCacheAbstractFullApiSelfTest /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), testName, true)); + U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(), testName, true)); super.beforeTestsStarted(); @@ -156,7 +156,7 @@ public class GridActivateExtensionTest extends GridCacheAbstractFullApiSelfTest condId = 0; - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), testName, true)); + U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(), testName, true)); } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/reconnect/IgniteAbstractStandByClientReconnectTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/reconnect/IgniteAbstractStandByClientReconnectTest.java index 0d9d9f2cc7dcc..4de1f34420e66 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/reconnect/IgniteAbstractStandByClientReconnectTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/reconnect/IgniteAbstractStandByClientReconnectTest.java @@ -44,7 +44,6 @@ import org.junit.Assert; import static org.apache.ignite.events.EventType.EVT_CLIENT_NODE_DISCONNECTED; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; /** * @@ -330,7 +329,7 @@ private AwaitDiscoverySpiListener( stopAllGrids(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, true)); + cleanPersistenceDir(); } @Override protected void afterTest() throws Exception { @@ -338,7 +337,7 @@ private AwaitDiscoverySpiListener( stopAllGrids(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, true)); + cleanPersistenceDir(); } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbAbstractTest.java index 9ce3077beed30..0823ee4fbfef9 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbAbstractTest.java @@ -171,7 +171,7 @@ protected boolean withClientNearCache() { /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); startGrids(gridCount()); @@ -204,7 +204,7 @@ protected boolean withClientNearCache() { stopAllGrids(); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbDynamicCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbDynamicCacheSelfTest.java index 00c22406ad961..eecf23ca6b383 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbDynamicCacheSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbDynamicCacheSelfTest.java @@ -31,8 +31,6 @@ import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; - /** * */ @@ -63,7 +61,7 @@ public class IgniteDbDynamicCacheSelfTest extends GridCommonAbstractTest { @Override protected void beforeTest() throws Exception { super.beforeTest(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbPutGetWithCacheStoreTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbPutGetWithCacheStoreTest.java index c76b6e2028d41..0fc311fd1f228 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbPutGetWithCacheStoreTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbPutGetWithCacheStoreTest.java @@ -79,14 +79,14 @@ public class IgniteDbPutGetWithCacheStoreTest extends GridCommonAbstractTest { /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); storeMap.clear(); } /** {@inheritDoc} */ @Override protected void afterTestsStopped() throws Exception { - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); storeMap.clear(); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/SwapPathConstructionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/SwapPathConstructionSelfTest.java index f22128c4d136f..416087b2b922f 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/SwapPathConstructionSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/SwapPathConstructionSelfTest.java @@ -68,11 +68,11 @@ public class SwapPathConstructionSelfTest extends GridCommonAbstractTest { private void cleanUpSwapDir() { Path relDir = Paths.get(U.getIgniteHome(), RELATIVE_SWAP_PATH); - deleteRecursively(relDir.toFile()); + U.delete(relDir.toFile()); Path absDir = Paths.get(getTmpDir(), ABSOLUTE_SWAP_PATH); - deleteRecursively(absDir.toFile()); + U.delete(absDir.toFile()); } /** 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 91a1144536bb8..b11482880afe5 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 @@ -106,9 +106,6 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; -import static org.springframework.util.FileSystemUtils.deleteRecursively; - /** * Utility class for tests. */ @@ -1967,13 +1964,6 @@ public static String randomString(Random rnd, int maxLen) { return b.toString(); } - /** - * @throws Exception If failed. - */ - public static void deleteDbFiles() throws Exception { - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); - } - /** * @param node Node. * @param topVer Ready exchange version to wait for before trying to merge exchanges. 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 e39d482f12866..d8cdda4660262 100755 --- 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 @@ -111,6 +111,7 @@ import static org.apache.ignite.cache.CacheMode.LOCAL; import static org.apache.ignite.cache.CacheRebalanceMode.NONE; import static org.apache.ignite.internal.processors.cache.GridCacheUtils.isNearEnabled; +import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ; @@ -1646,31 +1647,13 @@ protected static T doInTransaction(Ignite ignite, } /** - * @param file File or directory to delete. + * */ - protected boolean deleteRecursively(File file) { - boolean ok = true; - - long size = -1; - - if (file.isDirectory()) { - for (File f : file.listFiles()) - ok = deleteRecursively(f) & ok; - } - else - size = file.length(); - - if (!file.delete()) { - info("Failed to delete: " + file); - - ok = false; - } - - if (ok && log().isDebugEnabled()) // too much logging on real data - log().debug("Deleted OK: " + file.getAbsolutePath() + - (size >= 0 ? "(" + IgniteUtils.readableSize(size, false) + ")" : "")); - - return ok; + protected void cleanPersistenceDir() throws Exception { + U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(), "cp", false)); + U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(), "marshaller", false)); + U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(), "binary_meta", false)); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java b/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java index b119bfaaf91fd..c7850c6b4ebc0 100644 --- a/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java +++ b/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java @@ -45,16 +45,9 @@ protected File folder(String folder) throws IgniteCheckedException { return U.resolveWorkDirectory(U.defaultWorkDirectory(), folder, false); } - /** - * Delete all files created by database engine during test. - */ - private void cleanupDbFiles() throws IgniteCheckedException { - deleteRecursively(folder("db")); - } - /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { - cleanupDbFiles(); + cleanPersistenceDir(); stopAllGrids(); } @@ -63,7 +56,7 @@ private void cleanupDbFiles() throws IgniteCheckedException { @Override protected void afterTest() throws Exception { stopAllGrids(); - cleanupDbFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCheckClusterStateBeforeExecuteQueryTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCheckClusterStateBeforeExecuteQueryTest.java index f57a8c6ce534b..3a34fc73dc975 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCheckClusterStateBeforeExecuteQueryTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCheckClusterStateBeforeExecuteQueryTest.java @@ -24,13 +24,11 @@ import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteEx; -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 static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; import static org.apache.ignite.testframework.GridTestUtils.assertThrows; /** @@ -62,14 +60,14 @@ public class IgniteCheckClusterStateBeforeExecuteQueryTest extends GridCommonAbs @Override protected void beforeTest() throws Exception { stopAllGrids(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { stopAllGrids(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); } /** diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteErrorOnRebalanceTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteErrorOnRebalanceTest.java index c4061ede531cb..a9f927bde1d78 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteErrorOnRebalanceTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteErrorOnRebalanceTest.java @@ -89,7 +89,7 @@ public class IgniteErrorOnRebalanceTest extends GridCommonAbstractTest { @Override protected void afterTest() throws Exception { stopAllGrids(); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); super.afterTest(); } diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgnitePdsSingleNodeWithIndexingPutGetPersistenceTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgnitePdsSingleNodeWithIndexingPutGetPersistenceTest.java index 4a32dfdd50ef3..22c8851747306 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgnitePdsSingleNodeWithIndexingPutGetPersistenceTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgnitePdsSingleNodeWithIndexingPutGetPersistenceTest.java @@ -21,9 +21,6 @@ import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.configuration.WALMode; import org.apache.ignite.internal.processors.database.IgniteDbSingleNodeWithIndexingPutGetTest; -import org.apache.ignite.internal.util.typedef.internal.U; - -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; /** * @@ -43,7 +40,7 @@ public class IgnitePdsSingleNodeWithIndexingPutGetPersistenceTest extends Ignite /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); super.beforeTestsStarted(); } @@ -52,6 +49,6 @@ public class IgnitePdsSingleNodeWithIndexingPutGetPersistenceTest extends Ignite @Override protected void afterTestsStopped() throws Exception { super.afterTestsStopped(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); } } diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/LongIndexNameTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/LongIndexNameTest.java index ab0d520b5abdf..3c2b7133bb217 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/LongIndexNameTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/LongIndexNameTest.java @@ -28,15 +28,12 @@ import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.configuration.DataStorageConfiguration; -import org.apache.ignite.internal.util.typedef.internal.U; 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; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; - /** * */ @@ -54,14 +51,14 @@ public class LongIndexNameTest extends GridCommonAbstractTest { @Override protected void beforeTest() throws Exception { super.beforeTest(); - deleteWorkFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { super.afterTest(); - deleteWorkFiles(); + cleanPersistenceDir(); } /** @@ -150,13 +147,6 @@ public static List getIndexCfg() { return entities; } - /** - * @throws IgniteCheckedException If failed. - */ - private void deleteWorkFiles() throws IgniteCheckedException { - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); - } - /** * */ diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/OptimizedMarshallerIndexNameTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/OptimizedMarshallerIndexNameTest.java index 8e3dc5c0d3956..a288acca8b1fa 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/OptimizedMarshallerIndexNameTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/OptimizedMarshallerIndexNameTest.java @@ -101,7 +101,7 @@ protected static CacheConfiguration cacheConfiguration(String na /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), workSubdir(), true)); + U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(), workSubdir(), true)); startGrid(getTestIgniteInstanceName()); grid().active(true); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbSingleNodeWithIndexingWalRestoreTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbSingleNodeWithIndexingWalRestoreTest.java index 54667dfbe5aba..2d104aeac676d 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbSingleNodeWithIndexingWalRestoreTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbSingleNodeWithIndexingWalRestoreTest.java @@ -34,11 +34,8 @@ import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager; import org.apache.ignite.internal.util.typedef.F; -import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; - /** * Test verifies that binary metadata of values stored in cache and indexes upon these values * is handled correctly on cluster restart when persistent store is enabled and compact footer is turned on. @@ -193,13 +190,13 @@ public void testRegularClassesRestored() throws Exception { /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { stopAllGrids(); - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); + cleanPersistenceDir(); } } diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgnitePersistentStoreQueryWithMultipleClassesPerCacheTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgnitePersistentStoreQueryWithMultipleClassesPerCacheTest.java index c37dbdaedc9a2..b4d09468f534d 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgnitePersistentStoreQueryWithMultipleClassesPerCacheTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgnitePersistentStoreQueryWithMultipleClassesPerCacheTest.java @@ -29,14 +29,12 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.internal.IgniteEx; -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 static org.apache.ignite.IgniteSystemProperties.IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; /** * @@ -92,14 +90,14 @@ private CacheConfiguration cacheCfg(String name) { stopAllGrids(); - deleteWorkFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { stopAllGrids(); - deleteWorkFiles(); + cleanPersistenceDir(); System.clearProperty(IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK); } @@ -123,13 +121,6 @@ public void testSimple() throws Exception { } - /** - * @throws IgniteCheckedException On error. - */ - private void deleteWorkFiles() throws IgniteCheckedException { - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); - } - /** * */ diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgnitePersistentStoreSchemaLoadTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgnitePersistentStoreSchemaLoadTest.java index 2b7c5c2748a89..0817cae4b1444 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgnitePersistentStoreSchemaLoadTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgnitePersistentStoreSchemaLoadTest.java @@ -36,14 +36,12 @@ import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager; import org.apache.ignite.internal.processors.query.QuerySchema; import org.apache.ignite.internal.processors.query.QueryUtils; -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 static org.apache.ignite.IgniteSystemProperties.IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; /** * @@ -124,14 +122,14 @@ private CacheConfiguration cacheCfg(String name) { stopAllGrids(); - deleteWorkFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { stopAllGrids(); - deleteWorkFiles(); + cleanPersistenceDir(); System.clearProperty(IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK); } @@ -249,13 +247,6 @@ private int colsCnt(IgniteEx node, String cacheName) { return cnt; } - /** - * - */ - private void deleteWorkFiles() throws IgniteCheckedException { - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); - } - /** * @param node Node whose checkpoint to wait for. * @return Latch released when checkpoint happens. diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/baseline/IgniteBaselineLockPartitionOnAffinityRunAtomicCacheTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/baseline/IgniteBaselineLockPartitionOnAffinityRunAtomicCacheTest.java index c7663ea2ca3fb..8b5f90fad20e2 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/baseline/IgniteBaselineLockPartitionOnAffinityRunAtomicCacheTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/baseline/IgniteBaselineLockPartitionOnAffinityRunAtomicCacheTest.java @@ -54,12 +54,12 @@ public class IgniteBaselineLockPartitionOnAffinityRunAtomicCacheTest extends Ign /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); int gridCnt = gridCount(); @@ -84,13 +84,13 @@ public class IgniteBaselineLockPartitionOnAffinityRunAtomicCacheTest extends Ign stopAllGrids(); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void afterTestsStopped() throws Exception { stopAllGrids(); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } } diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/baseline/IgniteBaselineLockPartitionOnAffinityRunTxCacheTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/baseline/IgniteBaselineLockPartitionOnAffinityRunTxCacheTest.java index 88128296ec84a..b10ffb211defa 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/baseline/IgniteBaselineLockPartitionOnAffinityRunTxCacheTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/baseline/IgniteBaselineLockPartitionOnAffinityRunTxCacheTest.java @@ -45,12 +45,12 @@ public class IgniteBaselineLockPartitionOnAffinityRunTxCacheTest extends IgniteC /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); int gridCnt = gridCount(); @@ -75,13 +75,13 @@ public class IgniteBaselineLockPartitionOnAffinityRunTxCacheTest extends IgniteC stopAllGrids(); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void afterTestsStopped() throws Exception { stopAllGrids(); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } } diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/baseline/IgniteChangingBaselineCacheQueryNodeRestartSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/baseline/IgniteChangingBaselineCacheQueryNodeRestartSelfTest.java index 357d8e68562e7..8e049acc7d4fd 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/baseline/IgniteChangingBaselineCacheQueryNodeRestartSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/baseline/IgniteChangingBaselineCacheQueryNodeRestartSelfTest.java @@ -29,7 +29,6 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryNodeRestartSelfTest; -import org.apache.ignite.testframework.GridTestUtils; /** * @@ -53,7 +52,7 @@ public class IgniteChangingBaselineCacheQueryNodeRestartSelfTest extends IgniteC /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); startGrids(gridCount()); @@ -68,7 +67,7 @@ public class IgniteChangingBaselineCacheQueryNodeRestartSelfTest extends IgniteC @Override protected void afterTestsStopped() throws Exception { stopAllGrids(); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/baseline/IgniteStableBaselineBinObjFieldsQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/baseline/IgniteStableBaselineBinObjFieldsQuerySelfTest.java index e9d1b7953824a..2ccfa1bf2bd4f 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/baseline/IgniteStableBaselineBinObjFieldsQuerySelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/baseline/IgniteStableBaselineBinObjFieldsQuerySelfTest.java @@ -20,7 +20,6 @@ import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.processors.cache.IgniteBinaryObjectFieldsQuerySelfTest; -import org.apache.ignite.testframework.GridTestUtils; /** * @@ -44,14 +43,14 @@ public class IgniteStableBaselineBinObjFieldsQuerySelfTest extends IgniteBinaryO /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); initExtClassLoader(); } /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); startGrids(GRID_CNT + 1); @@ -66,13 +65,13 @@ public class IgniteStableBaselineBinObjFieldsQuerySelfTest extends IgniteBinaryO @Override protected void afterTest() throws Exception { stopAllGrids(); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void afterTestsStopped() throws Exception { super.afterTestsStopped(); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } } diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/baseline/IgniteStableBaselineCacheQueryNodeRestartsSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/baseline/IgniteStableBaselineCacheQueryNodeRestartsSelfTest.java index 849bbb6090522..81d803884e374 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/baseline/IgniteStableBaselineCacheQueryNodeRestartsSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/baseline/IgniteStableBaselineCacheQueryNodeRestartsSelfTest.java @@ -44,7 +44,7 @@ public class IgniteStableBaselineCacheQueryNodeRestartsSelfTest extends IgniteCa /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); startGrids(gridCount() + 1); @@ -63,6 +63,6 @@ public class IgniteStableBaselineCacheQueryNodeRestartsSelfTest extends IgniteCa @Override protected void afterTestsStopped() throws Exception { stopAllGrids(); - GridTestUtils.deleteDbFiles(); + cleanPersistenceDir(); } } From 2e92e0094b270aa8489e66d94bfcf15eadabfb4f Mon Sep 17 00:00:00 2001 From: EdShangGG Date: Mon, 12 Feb 2018 21:44:10 +0300 Subject: [PATCH 016/314] IGNITE-7626 Unify code in test which clean up persistence directories - Fixes #3512. Signed-off-by: Alexey Goncharuk (cherry picked from commit 6f6f8dd) --- .../IgnitePdsCacheDestroyDuringCheckpointTest.java | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCacheDestroyDuringCheckpointTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCacheDestroyDuringCheckpointTest.java index 72f73aaadb8d5..fba62dff3b8c3 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCacheDestroyDuringCheckpointTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCacheDestroyDuringCheckpointTest.java @@ -80,7 +80,7 @@ private DataStorageConfiguration createDbConfig() { @Override protected void beforeTestsStarted() throws Exception { super.beforeTestsStarted(); - deleteWorkFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @@ -89,7 +89,7 @@ private DataStorageConfiguration createDbConfig() { stopAllGrids(); - deleteWorkFiles(); + cleanPersistenceDir(); } /** @@ -151,11 +151,4 @@ private void checkCacheSizes(Ignite client) { } } } - - /** - * @throws IgniteCheckedException If fail. - */ - private void deleteWorkFiles() throws IgniteCheckedException { - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false)); - } } \ No newline at end of file From 3f86c127c78065999663a4fc4eaedb5e5d4bee1c Mon Sep 17 00:00:00 2001 From: EdShangGG Date: Mon, 12 Feb 2018 21:26:31 +0300 Subject: [PATCH 017/314] compilation fix --- .../persistence/db/file/IgnitePdsThreadInterruptionTest.java | 1 + 1 file changed, 1 insertion(+) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsThreadInterruptionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsThreadInterruptionTest.java index 0f33d55e0d4ef..c01c872620fb1 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsThreadInterruptionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsThreadInterruptionTest.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.processors.cache.persistence.db.file; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicReference; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; From 0b9322c566f9b464291854142ac02495bd1817e4 Mon Sep 17 00:00:00 2001 From: gg-shq Date: Wed, 7 Feb 2018 14:28:04 +0300 Subject: [PATCH 018/314] IGNITE-6917: Implemented SQL COPY command. --- .../internal/jdbc2/JdbcBulkLoadSelfTest.java | 185 ++++++ .../jdbc/JdbcErrorsAbstractSelfTest.java | 2 +- .../jdbc/suite/IgniteJdbcDriverTestSuite.java | 14 + .../JdbcThinBulkLoadAbstractSelfTest.java | 601 ++++++++++++++++++ ...BulkLoadAtomicPartitionedNearSelfTest.java | 39 ++ ...ThinBulkLoadAtomicPartitionedSelfTest.java | 39 ++ ...cThinBulkLoadAtomicReplicatedSelfTest.java | 39 ++ ...dTransactionalPartitionedNearSelfTest.java | 39 ++ ...kLoadTransactionalPartitionedSelfTest.java | 39 ++ ...lkLoadTransactionalReplicatedSelfTest.java | 39 ++ .../JdbcThinDynamicIndexAbstractSelfTest.java | 1 - .../clients/src/test/resources/bulkload0.csv | 0 .../clients/src/test/resources/bulkload1.csv | 1 + .../clients/src/test/resources/bulkload2.csv | 2 + .../src/test/resources/bulkload2_utf.csv | 2 + .../cache/query/BulkLoadContextCursor.java | 97 +++ .../internal/jdbc/thin/JdbcThinStatement.java | 68 +- .../ignite/internal/jdbc2/JdbcQueryTask.java | 12 +- .../bulkload/BulkLoadAckClientParameters.java | 92 +++ .../bulkload/BulkLoadCacheWriter.java | 31 + .../bulkload/BulkLoadCsvFormat.java | 159 +++++ .../bulkload/BulkLoadCsvParser.java | 65 ++ .../processors/bulkload/BulkLoadFormat.java | 33 + .../processors/bulkload/BulkLoadParser.java | 61 ++ .../bulkload/BulkLoadProcessor.java | 104 +++ .../bulkload/BulkLoadStreamerWriter.java | 65 ++ .../pipeline/CharsetDecoderBlock.java | 132 ++++ .../pipeline/CsvLineProcessorBlock.java | 70 ++ .../bulkload/pipeline/LineSplitterBlock.java | 72 +++ .../bulkload/pipeline/PipelineBlock.java | 66 ++ .../pipeline/StrListAppenderBlock.java | 52 ++ .../odbc/jdbc/JdbcBulkLoadAckResult.java | 111 ++++ .../odbc/jdbc/JdbcBulkLoadBatchRequest.java | 183 ++++++ .../odbc/jdbc/JdbcBulkLoadProcessor.java | 144 +++++ .../processors/odbc/jdbc/JdbcRequest.java | 7 + .../odbc/jdbc/JdbcRequestHandler.java | 90 ++- .../processors/odbc/jdbc/JdbcResult.java | 8 + .../ignite/internal/sql/SqlKeyword.java | 15 + .../apache/ignite/internal/sql/SqlParser.java | 18 +- .../sql/command/SqlBulkLoadCommand.java | 273 ++++++++ .../sql/SqlParserBulkLoadSelfTest.java | 70 ++ .../query/h2/DmlStatementsProcessor.java | 99 +++ .../processors/query/h2/IgniteH2Indexing.java | 35 +- .../query/h2/ddl/DdlStatementsProcessor.java | 2 + .../processors/query/h2/dml/UpdateMode.java | 11 +- .../processors/query/h2/dml/UpdatePlan.java | 20 +- .../query/h2/dml/UpdatePlanBuilder.java | 86 +++ .../IgniteCacheQuerySelfTestSuite.java | 2 + parent/pom.xml | 3 +- 49 files changed, 3361 insertions(+), 37 deletions(-) create mode 100644 modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcBulkLoadSelfTest.java create mode 100644 modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBulkLoadAbstractSelfTest.java create mode 100644 modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBulkLoadAtomicPartitionedNearSelfTest.java create mode 100644 modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBulkLoadAtomicPartitionedSelfTest.java create mode 100644 modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBulkLoadAtomicReplicatedSelfTest.java create mode 100644 modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBulkLoadTransactionalPartitionedNearSelfTest.java create mode 100644 modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBulkLoadTransactionalPartitionedSelfTest.java create mode 100644 modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBulkLoadTransactionalReplicatedSelfTest.java create mode 100644 modules/clients/src/test/resources/bulkload0.csv create mode 100644 modules/clients/src/test/resources/bulkload1.csv create mode 100644 modules/clients/src/test/resources/bulkload2.csv create mode 100644 modules/clients/src/test/resources/bulkload2_utf.csv create mode 100644 modules/core/src/main/java/org/apache/ignite/cache/query/BulkLoadContextCursor.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadAckClientParameters.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadCacheWriter.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadCsvFormat.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadCsvParser.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadFormat.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadParser.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadProcessor.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadStreamerWriter.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/pipeline/CharsetDecoderBlock.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/pipeline/CsvLineProcessorBlock.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/pipeline/LineSplitterBlock.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/pipeline/PipelineBlock.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/pipeline/StrListAppenderBlock.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBulkLoadAckResult.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBulkLoadBatchRequest.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBulkLoadProcessor.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/sql/command/SqlBulkLoadCommand.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/sql/SqlParserBulkLoadSelfTest.java diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcBulkLoadSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcBulkLoadSelfTest.java new file mode 100644 index 0000000000000..d9506cf718ffc --- /dev/null +++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcBulkLoadSelfTest.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.internal.jdbc2; + +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.cache.query.annotations.QuerySqlField; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.ConnectorConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +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.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +import java.io.Serializable; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.Collections; +import java.util.Properties; +import java.util.concurrent.Callable; + +import static org.apache.ignite.IgniteJdbcDriver.CFG_URL_PREFIX; +import static org.apache.ignite.cache.CacheMode.PARTITIONED; +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; + +/** COPY command test for the regular JDBC driver. */ +public class JdbcBulkLoadSelfTest extends GridCommonAbstractTest { + /** JDBC URL. */ + private static final String BASE_URL = CFG_URL_PREFIX + + "cache=default@modules/clients/src/test/config/jdbc-config.xml"; + + /** Connection. */ + protected Connection conn; + + /** The logger. */ + protected transient IgniteLogger log; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + return getConfiguration0(gridName); + } + + /** + * @param gridName Grid name. + * @return Grid configuration used for starting the grid. + * @throws Exception If failed. + */ + private IgniteConfiguration getConfiguration0(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + CacheConfiguration cache = defaultCacheConfiguration(); + + cache.setCacheMode(PARTITIONED); + cache.setBackups(1); + cache.setWriteSynchronizationMode(FULL_SYNC); + cache.setIndexedTypes( + Integer.class, Person.class + ); + + cfg.setCacheConfiguration(cache); + cfg.setLocalHost("127.0.0.1"); + + TcpDiscoverySpi disco = new TcpDiscoverySpi(); + + TcpDiscoveryVmIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); + ipFinder.setAddresses(Collections.singleton("127.0.0.1:47500..47501")); + + disco.setIpFinder(ipFinder); + + cfg.setDiscoverySpi(disco); + + cfg.setConnectorConfiguration(new ConnectorConfiguration()); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + startGrids(2); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + } + + /** + * Establishes the JDBC connection. + * + * @return Connection to use for the test. + * @throws Exception if failed. + */ + private Connection createConnection() throws Exception { + Properties props = new Properties(); + + return DriverManager.getConnection(BASE_URL, props); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + U.closeQuiet(conn); + + ignite(0).cache(DEFAULT_CACHE_NAME).clear(); + + super.afterTest(); + } + + /** + * This is more a placeholder for implementation of IGNITE-7553. + * + * @throws Exception if failed. + */ + public void testBulkLoadThrows() throws Exception { + GridTestUtils.assertThrows(null, new Callable() { + @Override public Object call() throws Exception { + conn = createConnection(); + + try (Statement stmt = conn.createStatement()) { + stmt.executeUpdate("copy from \"dummy.csv\" into Person" + + " (_key, id, firstName, lastName) format csv"); + + return null; + } + } + }, SQLException.class, "COPY command is currently supported only in thin JDBC driver."); + } + + /** + * A test class for creating a query entity. + */ + @SuppressWarnings("UnusedDeclaration") + private static class Person implements Serializable { + /** ID. */ + @QuerySqlField + private final int id; + + /** First name. */ + @QuerySqlField(index = false) + private final String firstName; + + /** Last name. */ + @QuerySqlField(index = false) + private final String lastName; + + /** Age. */ + @QuerySqlField + private final int age; + + /** + * @param id ID. + * @param firstName First name + * @param lastName Last name + * @param age Age. + */ + private Person(int id, String firstName, String lastName, int age) { + assert !F.isEmpty(firstName); + assert !F.isEmpty(lastName); + assert age > 0; + + this.id = id; + this.firstName = firstName; + this.lastName = lastName; + this.age = age; + } + } +} diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcErrorsAbstractSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcErrorsAbstractSelfTest.java index 49746b690eed1..20594083ce82e 100644 --- a/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcErrorsAbstractSelfTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcErrorsAbstractSelfTest.java @@ -107,7 +107,7 @@ public void testIndexErrors() throws SQLException { */ public void testDmlErrors() throws SQLException { checkErrorState("INSERT INTO \"test\".INTEGER(_key, _val) values(1, null)", "22004", - "Value for INSERT, MERGE, or UPDATE must not be null"); + "Value for INSERT, COPY, MERGE, or UPDATE must not be null"); checkErrorState("INSERT INTO \"test\".INTEGER(_key, _val) values(1, 'zzz')", "0700B", "Value conversion failed [from=java.lang.String, to=java.lang.Integer]"); diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java index ff4d69f14c702..656e218ecee09 100644 --- a/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java +++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java @@ -35,6 +35,12 @@ import org.apache.ignite.jdbc.JdbcStatementSelfTest; import org.apache.ignite.jdbc.thin.JdbcThinAutoCloseServerCursorTest; import org.apache.ignite.jdbc.thin.JdbcThinBatchSelfTest; +import org.apache.ignite.jdbc.thin.JdbcThinBulkLoadAtomicPartitionedNearSelfTest; +import org.apache.ignite.jdbc.thin.JdbcThinBulkLoadAtomicPartitionedSelfTest; +import org.apache.ignite.jdbc.thin.JdbcThinBulkLoadAtomicReplicatedSelfTest; +import org.apache.ignite.jdbc.thin.JdbcThinBulkLoadTransactionalPartitionedNearSelfTest; +import org.apache.ignite.jdbc.thin.JdbcThinBulkLoadTransactionalPartitionedSelfTest; +import org.apache.ignite.jdbc.thin.JdbcThinBulkLoadTransactionalReplicatedSelfTest; import org.apache.ignite.jdbc.thin.JdbcThinComplexDmlDdlSelfTest; import org.apache.ignite.jdbc.thin.JdbcThinComplexQuerySelfTest; import org.apache.ignite.jdbc.thin.JdbcThinConnectionSelfTest; @@ -154,6 +160,14 @@ public static TestSuite suite() throws Exception { suite.addTest(new TestSuite(JdbcThinDynamicIndexTransactionalPartitionedSelfTest.class)); suite.addTest(new TestSuite(JdbcThinDynamicIndexTransactionalReplicatedSelfTest.class)); + // New thin JDBC driver, DML tests + suite.addTest(new TestSuite(JdbcThinBulkLoadAtomicPartitionedNearSelfTest.class)); + suite.addTest(new TestSuite(JdbcThinBulkLoadAtomicPartitionedSelfTest.class)); + suite.addTest(new TestSuite(JdbcThinBulkLoadAtomicReplicatedSelfTest.class)); + suite.addTest(new TestSuite(JdbcThinBulkLoadTransactionalPartitionedNearSelfTest.class)); + suite.addTest(new TestSuite(JdbcThinBulkLoadTransactionalPartitionedSelfTest.class)); + suite.addTest(new TestSuite(JdbcThinBulkLoadTransactionalReplicatedSelfTest.class)); + // New thin JDBC driver, full SQL tests suite.addTest(new TestSuite(JdbcThinComplexDmlDdlSelfTest.class)); diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBulkLoadAbstractSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBulkLoadAbstractSelfTest.java new file mode 100644 index 0000000000000..761f700632880 --- /dev/null +++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBulkLoadAbstractSelfTest.java @@ -0,0 +1,601 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.jdbc.thin; + +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.QueryEntity; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.NearCacheConfiguration; +import org.apache.ignite.internal.processors.query.QueryUtils; +import org.apache.ignite.testframework.GridTestUtils; + +import java.sql.BatchUpdateException; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.Collection; +import java.util.Collections; +import java.util.Objects; +import java.util.concurrent.Callable; + +import static org.apache.ignite.IgniteSystemProperties.IGNITE_SQL_PARSER_DISABLE_H2_FALLBACK; +import static org.apache.ignite.cache.CacheMode.PARTITIONED; +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; +import static org.apache.ignite.internal.util.IgniteUtils.resolveIgnitePath; + +/** + * COPY statement tests. + */ +public abstract class JdbcThinBulkLoadAbstractSelfTest extends JdbcThinAbstractDmlStatementSelfTest { + /** Default table name. */ + private static final String TBL_NAME = "Person"; + + /** JDBC statement. */ + private Statement stmt; + + /** A CSV file with zero records */ + private static final String BULKLOAD_EMPTY_CSV_FILE = + Objects.requireNonNull(resolveIgnitePath("/modules/clients/src/test/resources/bulkload0.csv")) + .getAbsolutePath(); + + /** A CSV file with one record. */ + private static final String BULKLOAD_ONE_LINE_CSV_FILE = + Objects.requireNonNull(resolveIgnitePath("/modules/clients/src/test/resources/bulkload1.csv")) + .getAbsolutePath(); + + /** A CSV file with two records. */ + private static final String BULKLOAD_TWO_LINES_CSV_FILE = + Objects.requireNonNull(resolveIgnitePath("/modules/clients/src/test/resources/bulkload2.csv")) + .getAbsolutePath(); + + /** A file with UTF records. */ + private static final String BULKLOAD_UTF_CSV_FILE = + Objects.requireNonNull(resolveIgnitePath("/modules/clients/src/test/resources/bulkload2_utf.csv")) + .getAbsolutePath(); + + /** Basic COPY statement used in majority of the tests. */ + public static final String BASIC_SQL_COPY_STMT = + "copy from \"" + BULKLOAD_TWO_LINES_CSV_FILE + "\"" + + " into " + TBL_NAME + + " (_key, age, firstName, lastName)" + + " format csv"; + + /** {@inheritDoc} */ + @Override protected CacheConfiguration cacheConfig() { + return cacheConfigWithIndexedTypes(); + } + + /** + * Creates cache configuration with {@link QueryEntity} created + * using {@link CacheConfiguration#setIndexedTypes(Class[])} call. + * + * @return The cache configuration. + */ + @SuppressWarnings("unchecked") + private CacheConfiguration cacheConfigWithIndexedTypes() { + CacheConfiguration cache = defaultCacheConfiguration(); + + cache.setCacheMode(cacheMode()); + cache.setAtomicityMode(atomicityMode()); + cache.setWriteSynchronizationMode(FULL_SYNC); + + if (cacheMode() == PARTITIONED) + cache.setBackups(1); + + if (nearCache()) + cache.setNearConfiguration(new NearCacheConfiguration()); + + cache.setIndexedTypes( + String.class, Person.class + ); + + return cache; + } + + /** + * Returns true if we are testing near cache. + * + * @return true if we are testing near cache. + */ + protected abstract boolean nearCache(); + + /** + * Returns cache atomicity mode we are testing. + * + * @return The cache atomicity mode we are testing. + */ + protected abstract CacheAtomicityMode atomicityMode(); + + /** + * Returns cache mode we are testing. + * + * @return The cache mode we are testing. + */ + protected abstract CacheMode cacheMode(); + + /** + * Creates cache configuration with {@link QueryEntity} created + * using {@link CacheConfiguration#setQueryEntities(Collection)} call. + * + * @return The cache configuration. + */ + private CacheConfiguration cacheConfigWithQueryEntity() { + CacheConfiguration cache = defaultCacheConfiguration(); + + cache.setCacheMode(PARTITIONED); + cache.setBackups(1); + cache.setWriteSynchronizationMode(FULL_SYNC); + + QueryEntity e = new QueryEntity(); + + e.setKeyType(String.class.getName()); + e.setValueType("Person"); + + e.addQueryField("id", Integer.class.getName(), null); + e.addQueryField("age", Integer.class.getName(), null); + e.addQueryField("firstName", String.class.getName(), null); + e.addQueryField("lastName", String.class.getName(), null); + + cache.setQueryEntities(Collections.singletonList(e)); + + return cache; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + System.setProperty(IGNITE_SQL_PARSER_DISABLE_H2_FALLBACK, "TRUE"); + + stmt = conn.createStatement(); + + assertNotNull(stmt); + assertFalse(stmt.isClosed()); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + if (stmt != null && !stmt.isClosed()) + stmt.close(); + + assertTrue(stmt.isClosed()); + + System.clearProperty(IGNITE_SQL_PARSER_DISABLE_H2_FALLBACK); + + super.afterTest(); + } + + /** + * Dead-on-arrival test. Imports two-entry CSV file into a table and checks + * the created entries using SELECT statement. + * + * @throws SQLException If failed. + */ + public void testBasicStatement() throws SQLException { + int updatesCnt = stmt.executeUpdate(BASIC_SQL_COPY_STMT); + + assertEquals(2, updatesCnt); + + checkCacheContents(TBL_NAME, true, 2); + } + + /** + * Imports two-entry CSV file with UTF-8 characters into a table and checks + * the created entries using SELECT statement. + * + * @throws SQLException If failed. + */ + public void testUtf() throws SQLException { + int updatesCnt = stmt.executeUpdate( + "copy from \"" + BULKLOAD_UTF_CSV_FILE + "\" into " + TBL_NAME + + " (_key, age, firstName, lastName)" + + " format csv"); + + assertEquals(2, updatesCnt); + + checkUtfCacheContents(TBL_NAME, true, 2); + } + + /** + * Imports two-entry CSV file with UTF-8 characters into a table using batch size of one byte + * (thus splitting each two-byte UTF-8 character into two batches) + * and checks the created entries using SELECT statement. + * + * @throws SQLException If failed. + */ + public void testUtfBatchSize_1() throws SQLException { + int updatesCnt = stmt.executeUpdate( + "copy from \"" + BULKLOAD_UTF_CSV_FILE + "\" into " + TBL_NAME + + " (_key, age, firstName, lastName)" + + " format csv batch_size 1"); + + assertEquals(2, updatesCnt); + + checkUtfCacheContents(TBL_NAME, true, 2); + } + + /** + * Imports one-entry CSV file into a table and checks the entry created using SELECT statement. + * + * @throws SQLException If failed. + */ + public void testOneLineFile() throws SQLException { + int updatesCnt = stmt.executeUpdate( + "copy from \"" + BULKLOAD_ONE_LINE_CSV_FILE + "\" into " + TBL_NAME + + " (_key, age, firstName, lastName)" + + " format csv"); + + assertEquals(1, updatesCnt); + + checkCacheContents(TBL_NAME, true, 1); + } + + /** + * Imports zero-entry CSV file into a table and checks that no entries are created + * using SELECT statement. + * + * @throws SQLException If failed. + */ + public void testEmptyFile() throws SQLException { + int updatesCnt = stmt.executeUpdate( + "copy from \"" + BULKLOAD_EMPTY_CSV_FILE + "\" into " + TBL_NAME + + " (_key, age, firstName, lastName)" + + " format csv"); + + assertEquals(0, updatesCnt); + + checkCacheContents(TBL_NAME, true, 0); + } + + /** + * Checks that error is reported for a non-existent file. + */ + public void testWrongFileName() { + GridTestUtils.assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + stmt.executeUpdate( + "copy from \"nonexistent\" into Person" + + " (_key, age, firstName, lastName)" + + " format csv"); + + return null; + } + }, SQLException.class, "Failed to read file: 'nonexistent'"); + } + + /** + * Checks that error is reported if the destination table is missing. + */ + public void testMissingTable() { + GridTestUtils.assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + stmt.executeUpdate( + "copy from \"" + BULKLOAD_TWO_LINES_CSV_FILE + "\" into Peterson" + + " (_key, age, firstName, lastName)" + + " format csv"); + + return null; + } + }, SQLException.class, "Table does not exist: PETERSON"); + } + + /** + * Checks that error is reported when a non-existing column is specified in the SQL command. + */ + public void testWrongColumnName() { + GridTestUtils.assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + stmt.executeUpdate( + "copy from \"" + BULKLOAD_TWO_LINES_CSV_FILE + "\" into Person" + + " (_key, age, firstName, lostName)" + + " format csv"); + + return null; + } + }, SQLException.class, "Column \"LOSTNAME\" not found"); + } + + /** + * Checks that error is reported if field read from CSV file cannot be converted to the type of the column. + */ + public void testWrongColumnType() { + GridTestUtils.assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + stmt.executeUpdate( + "copy from \"" + BULKLOAD_TWO_LINES_CSV_FILE + "\" into Person" + + " (_key, firstName, age, lastName)" + + " format csv"); + + return null; + } + }, SQLException.class, "Value conversion failed [from=java.lang.String, to=java.lang.Integer]"); + } + + /** + * Checks that if even a subset of fields is imported, the imported fields are set correctly. + * + * @throws SQLException If failed. + */ + public void testFieldsSubset() throws SQLException { + int updatesCnt = stmt.executeUpdate( + "copy from \"" + BULKLOAD_TWO_LINES_CSV_FILE + "\" into " + TBL_NAME + + " (_key, age, firstName)" + + " format csv"); + + assertEquals(2, updatesCnt); + + checkCacheContents(TBL_NAME, false, 2); + } + + /** + * Checks that bulk load works when we create table using 'CREATE TABLE' command. + * + * The majority of the tests in this class use {@link CacheConfiguration#setIndexedTypes(Class[])} + * to create the table. + * + * @throws SQLException If failed. + */ + public void testCreateAndBulkLoadTable() throws SQLException { + String tblName = QueryUtils.DFLT_SCHEMA + ".\"PersonTbl\""; + + execute(conn, "create table " + tblName + + " (id int primary key, age int, firstName varchar(30), lastName varchar(30))"); + + int updatesCnt = stmt.executeUpdate( + "copy from \"" + BULKLOAD_TWO_LINES_CSV_FILE + "\" into " + tblName + + "(_key, age, firstName, lastName)" + + " format csv"); + + assertEquals(2, updatesCnt); + + checkCacheContents(tblName, true, 2); + } + + /** + * Checks that bulk load works when we create table with {@link CacheConfiguration#setQueryEntities(Collection)}. + * + * The majority of the tests in this class use {@link CacheConfiguration#setIndexedTypes(Class[])} + * to create a table. + * + * @throws SQLException If failed. + */ + @SuppressWarnings("unchecked") + public void testConfigureQueryEntityAndBulkLoad() throws SQLException { + ignite(0).getOrCreateCache(cacheConfigWithQueryEntity()); + + int updatesCnt = stmt.executeUpdate(BASIC_SQL_COPY_STMT); + + assertEquals(2, updatesCnt); + + checkCacheContents(TBL_NAME, true, 2); + } + + /** + * Checks that bulk load works when we use batch size of 1 byte and thus + * create multiple batches per COPY. + * + * @throws SQLException If failed. + */ + public void testBatchSize_1() throws SQLException { + int updatesCnt = stmt.executeUpdate(BASIC_SQL_COPY_STMT + " batch_size 1"); + + assertEquals(2, updatesCnt); + + checkCacheContents(TBL_NAME, true, 2); + } + + /** + * Verifies exception thrown if COPY is added into a batch. + * + * @throws SQLException If failed. + */ + public void testMultipleStatement() throws SQLException { + GridTestUtils.assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + stmt.addBatch(BASIC_SQL_COPY_STMT); + + stmt.addBatch("copy from \"" + BULKLOAD_ONE_LINE_CSV_FILE + "\" into " + TBL_NAME + + " (_key, age, firstName, lastName)" + + " format csv"); + + stmt.addBatch("copy from \"" + BULKLOAD_UTF_CSV_FILE + "\" into " + TBL_NAME + + " (_key, age, firstName, lastName)" + + " format csv"); + + stmt.executeBatch(); + + return null; + } + }, BatchUpdateException.class, "COPY command cannot be executed in batch mode."); + } + + /** + * Verifies that COPY command is rejected by Statement.executeQuery(). + * + * @throws SQLException If failed. + */ + public void testExecuteQuery() throws SQLException { + GridTestUtils.assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + stmt.executeQuery(BASIC_SQL_COPY_STMT); + + return null; + } + }, SQLException.class, "The query isn't SELECT query"); + } + + /** + * Verifies that COPY command works in Statement.execute(). + * + * @throws SQLException If failed. + */ + public void testExecute() throws SQLException { + boolean isRowSet = stmt.execute(BASIC_SQL_COPY_STMT); + + assertFalse(isRowSet); + + checkCacheContents(TBL_NAME, true, 2); + } + + /** + * Verifies that COPY command can be called with PreparedStatement.executeUpdate(). + * + * @throws SQLException If failed. + */ + public void testPreparedStatementWithExecuteUpdate() throws SQLException { + PreparedStatement pstmt = conn.prepareStatement(BASIC_SQL_COPY_STMT); + + int updatesCnt = pstmt.executeUpdate(); + + assertEquals(2, updatesCnt); + + checkCacheContents(TBL_NAME, true, 2); + } + + /** + * Verifies that COPY command reports an error when used with PreparedStatement parameter. + * + * @throws SQLException If failed. + */ + public void testPreparedStatementWithParameter() throws SQLException { + GridTestUtils.assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + PreparedStatement pstmt = conn.prepareStatement( + "copy from \"" + BULKLOAD_TWO_LINES_CSV_FILE + "\" into " + TBL_NAME + + " (_key, age, firstName, lastName)" + + " format ?"); + + pstmt.setString(1, "csv"); + + pstmt.executeUpdate(); + + return null; + } + }, SQLException.class, "Unexpected token: \"?\" (expected: \"[identifier]\""); + } + + /** + * Verifies that COPY command can be called with PreparedStatement.execute(). + * + * @throws SQLException If failed. + */ + public void testPreparedStatementWithExecute() throws SQLException { + PreparedStatement pstmt = conn.prepareStatement(BASIC_SQL_COPY_STMT); + + boolean isRowSet = pstmt.execute(); + + assertFalse(isRowSet); + + checkCacheContents(TBL_NAME, true, 2); + } + + /** + * Verifies that COPY command is rejected by PreparedStatement.executeQuery(). + */ + public void testPreparedStatementWithExecuteQuery() { + GridTestUtils.assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + PreparedStatement pstmt = conn.prepareStatement(BASIC_SQL_COPY_STMT); + + pstmt.executeQuery(); + + return null; + } + }, SQLException.class, "The query isn't SELECT query"); + } + + /** + * Checks cache contents for a typical test using SQL SELECT command. + * + * @param tblName Table name to query. + * @param checkLastName Check 'lastName' column (not imported in some tests). + * @param recCnt Number of records to expect. + * @throws SQLException When one of checks has failed. + */ + private void checkCacheContents(String tblName, boolean checkLastName, int recCnt) throws SQLException { + ResultSet rs = stmt.executeQuery("select _key, age, firstName, lastName from " + tblName); + + assert rs != null; + + int cnt = 0; + + while (rs.next()) { + int id = rs.getInt("_key"); + + if (id == 123) { + assertEquals(12, rs.getInt("age")); + assertEquals("FirstName123 MiddleName123", rs.getString("firstName")); + if (checkLastName) + assertEquals("LastName123", rs.getString("lastName")); + } + else if (id == 456) { + assertEquals(45, rs.getInt("age")); + assertEquals("FirstName456", rs.getString("firstName")); + if (checkLastName) + assertEquals("LastName456", rs.getString("lastName")); + } + else + fail("Wrong ID: " + id); + + cnt++; + } + + assertEquals(recCnt, cnt); + } + + /** + * Checks cache contents for a UTF-8 bulk load tests using SQL SELECT command. + * + * @param tblName Table name to query. + * @param checkLastName Check 'lastName' column (not imported in some tests). + * @param recCnt Number of records to expect. + * @throws SQLException When one of checks has failed. + */ + private void checkUtfCacheContents(String tblName, boolean checkLastName, int recCnt) throws SQLException { + ResultSet rs = stmt.executeQuery("select _key, age, firstName, lastName from " + tblName); + + assert rs != null; + + int cnt = 0; + + while (rs.next()) { + int id = rs.getInt("_key"); + + if (id == 123) { + assertEquals(12, rs.getInt("age")); + assertEquals("Имя123 Отчество123", rs.getString("firstName")); + if (checkLastName) + assertEquals("Фамилия123", rs.getString("lastName")); + } + else if (id == 456) { + assertEquals(45, rs.getInt("age")); + assertEquals("Имя456", rs.getString("firstName")); + if (checkLastName) + assertEquals("Фамилия456", rs.getString("lastName")); + } + else + fail("Wrong ID: " + id); + + cnt++; + } + + assertEquals(recCnt, cnt); + } +} diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBulkLoadAtomicPartitionedNearSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBulkLoadAtomicPartitionedNearSelfTest.java new file mode 100644 index 0000000000000..887b1d9d7eae0 --- /dev/null +++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBulkLoadAtomicPartitionedNearSelfTest.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.jdbc.thin; + +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheMode; + +/** A {@link JdbcThinBulkLoadAbstractSelfTest} for partitioned atomic near-cache mode. */ +public class JdbcThinBulkLoadAtomicPartitionedNearSelfTest extends JdbcThinBulkLoadAbstractSelfTest { + /** {@inheritDoc} */ + @Override protected CacheMode cacheMode() { + return CacheMode.PARTITIONED; + } + + /** {@inheritDoc} */ + @Override protected CacheAtomicityMode atomicityMode() { + return CacheAtomicityMode.ATOMIC; + } + + /** {@inheritDoc} */ + @Override protected boolean nearCache() { + return true; + } +} diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBulkLoadAtomicPartitionedSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBulkLoadAtomicPartitionedSelfTest.java new file mode 100644 index 0000000000000..55813330c3422 --- /dev/null +++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBulkLoadAtomicPartitionedSelfTest.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.jdbc.thin; + +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheMode; + +/** A {@link JdbcThinBulkLoadAbstractSelfTest} for partitioned atomic mode. */ +public class JdbcThinBulkLoadAtomicPartitionedSelfTest extends JdbcThinBulkLoadAbstractSelfTest { + /** {@inheritDoc} */ + @Override protected CacheMode cacheMode() { + return CacheMode.PARTITIONED; + } + + /** {@inheritDoc} */ + @Override protected CacheAtomicityMode atomicityMode() { + return CacheAtomicityMode.ATOMIC; + } + + /** {@inheritDoc} */ + @Override protected boolean nearCache() { + return false; + } +} diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBulkLoadAtomicReplicatedSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBulkLoadAtomicReplicatedSelfTest.java new file mode 100644 index 0000000000000..c3d69afbd89fa --- /dev/null +++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBulkLoadAtomicReplicatedSelfTest.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.jdbc.thin; + +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheMode; + +/** A {@link JdbcThinBulkLoadAbstractSelfTest} for replicated atomic near-cache mode. */ +public class JdbcThinBulkLoadAtomicReplicatedSelfTest extends JdbcThinBulkLoadAbstractSelfTest { + /** {@inheritDoc} */ + @Override protected CacheMode cacheMode() { + return CacheMode.REPLICATED; + } + + /** {@inheritDoc} */ + @Override protected CacheAtomicityMode atomicityMode() { + return CacheAtomicityMode.ATOMIC; + } + + /** {@inheritDoc} */ + @Override protected boolean nearCache() { + return false; + } +} diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBulkLoadTransactionalPartitionedNearSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBulkLoadTransactionalPartitionedNearSelfTest.java new file mode 100644 index 0000000000000..9336dd194788a --- /dev/null +++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBulkLoadTransactionalPartitionedNearSelfTest.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.jdbc.thin; + +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheMode; + +/** A {@link JdbcThinBulkLoadAbstractSelfTest} for partitioned transactional near-cache mode. */ +public class JdbcThinBulkLoadTransactionalPartitionedNearSelfTest extends JdbcThinBulkLoadAbstractSelfTest { + /** {@inheritDoc} */ + @Override protected CacheMode cacheMode() { + return CacheMode.PARTITIONED; + } + + /** {@inheritDoc} */ + @Override protected CacheAtomicityMode atomicityMode() { + return CacheAtomicityMode.TRANSACTIONAL; + } + + /** {@inheritDoc} */ + @Override protected boolean nearCache() { + return true; + } +} diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBulkLoadTransactionalPartitionedSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBulkLoadTransactionalPartitionedSelfTest.java new file mode 100644 index 0000000000000..d1dea2a7e302c --- /dev/null +++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBulkLoadTransactionalPartitionedSelfTest.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.jdbc.thin; + +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheMode; + +/** A {@link JdbcThinBulkLoadAbstractSelfTest} for partitioned transactional mode. */ +public class JdbcThinBulkLoadTransactionalPartitionedSelfTest extends JdbcThinBulkLoadAbstractSelfTest { + /** {@inheritDoc} */ + @Override protected CacheMode cacheMode() { + return CacheMode.PARTITIONED; + } + + /** {@inheritDoc} */ + @Override protected CacheAtomicityMode atomicityMode() { + return CacheAtomicityMode.TRANSACTIONAL; + } + + /** {@inheritDoc} */ + @Override protected boolean nearCache() { + return false; + } +} diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBulkLoadTransactionalReplicatedSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBulkLoadTransactionalReplicatedSelfTest.java new file mode 100644 index 0000000000000..1c377fac96b29 --- /dev/null +++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBulkLoadTransactionalReplicatedSelfTest.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.jdbc.thin; + +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheMode; + +/** A {@link JdbcThinBulkLoadAbstractSelfTest} for replicated transactional mode. */ +public class JdbcThinBulkLoadTransactionalReplicatedSelfTest extends JdbcThinBulkLoadAbstractSelfTest { + /** {@inheritDoc} */ + @Override protected CacheMode cacheMode() { + return CacheMode.REPLICATED; + } + + /** {@inheritDoc} */ + @Override protected CacheAtomicityMode atomicityMode() { + return CacheAtomicityMode.TRANSACTIONAL; + } + + /** {@inheritDoc} */ + @Override protected boolean nearCache() { + return false; + } +} diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinDynamicIndexAbstractSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinDynamicIndexAbstractSelfTest.java index dbe93a49f465d..539713aeb4cdb 100644 --- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinDynamicIndexAbstractSelfTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinDynamicIndexAbstractSelfTest.java @@ -25,7 +25,6 @@ import java.util.List; import java.util.concurrent.Callable; import org.apache.ignite.IgniteCache; -import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cache.CacheWriteSynchronizationMode; diff --git a/modules/clients/src/test/resources/bulkload0.csv b/modules/clients/src/test/resources/bulkload0.csv new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/modules/clients/src/test/resources/bulkload1.csv b/modules/clients/src/test/resources/bulkload1.csv new file mode 100644 index 0000000000000..596ac32aa66b0 --- /dev/null +++ b/modules/clients/src/test/resources/bulkload1.csv @@ -0,0 +1 @@ +123,12,"FirstName123 MiddleName123",LastName123 \ No newline at end of file diff --git a/modules/clients/src/test/resources/bulkload2.csv b/modules/clients/src/test/resources/bulkload2.csv new file mode 100644 index 0000000000000..d398c19c1cd9d --- /dev/null +++ b/modules/clients/src/test/resources/bulkload2.csv @@ -0,0 +1,2 @@ +123,12,"FirstName123 MiddleName123",LastName123 +456,45,"FirstName456","LastName456" \ No newline at end of file diff --git a/modules/clients/src/test/resources/bulkload2_utf.csv b/modules/clients/src/test/resources/bulkload2_utf.csv new file mode 100644 index 0000000000000..bdb64896be58f --- /dev/null +++ b/modules/clients/src/test/resources/bulkload2_utf.csv @@ -0,0 +1,2 @@ +123,12,"Имя123 Отчество123",Фамилия123 +456,45,"Имя456","Фамилия456" \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/BulkLoadContextCursor.java b/modules/core/src/main/java/org/apache/ignite/cache/query/BulkLoadContextCursor.java new file mode 100644 index 0000000000000..b7fdec3e8d07f --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/cache/query/BulkLoadContextCursor.java @@ -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. + */ + +package org.apache.ignite.cache.query; + +import org.apache.ignite.internal.processors.bulkload.BulkLoadAckClientParameters; +import org.apache.ignite.internal.processors.bulkload.BulkLoadProcessor; +import org.jetbrains.annotations.NotNull; + +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; + +/** + * A special FieldsQueryCursor subclass that is used as a sentinel to transfer data from bulk load + * (COPY) command to the JDBC or other client-facing driver: the bulk load batch processor + * and parameters to send to the client. + * */ +public class BulkLoadContextCursor implements FieldsQueryCursor> { + /** Bulk load context from SQL command. */ + private final BulkLoadProcessor processor; + + /** Bulk load parameters to send to the client. */ + private final BulkLoadAckClientParameters clientParams; + + /** + * Creates a cursor. + * + * @param processor Bulk load context object to store. + * @param clientParams Parameters to send to client. + */ + public BulkLoadContextCursor(BulkLoadProcessor processor, BulkLoadAckClientParameters clientParams) { + this.processor = processor; + this.clientParams = clientParams; + } + + /** + * Returns a bulk load context. + * + * @return a bulk load context. + */ + public BulkLoadProcessor bulkLoadProcessor() { + return processor; + } + + /** + * Returns the bulk load parameters to send to the client. + * + * @return The bulk load parameters to send to the client. + */ + public BulkLoadAckClientParameters clientParams() { + return clientParams; + } + + /** {@inheritDoc} */ + @Override public List> getAll() { + return Collections.singletonList(Arrays.asList(processor, clientParams)); + } + + /** {@inheritDoc} */ + @NotNull @Override public Iterator> iterator() { + return getAll().iterator(); + } + + /** {@inheritDoc} */ + @Override public void close() { + // no-op + } + + /** {@inheritDoc} */ + @Override public String getFieldName(int idx) { + if (idx < 0 || idx > 1) + throw new IndexOutOfBoundsException(); + + return idx == 0 ? "processor" : "clientParams"; + } + + /** {@inheritDoc} */ + @Override public int getColumnsCount() { + return 2; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java index d29df932c2ba5..202001149d932 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java @@ -17,6 +17,9 @@ package org.apache.ignite.internal.jdbc.thin; +import java.io.BufferedInputStream; +import java.io.FileInputStream; +import java.io.InputStream; import java.sql.BatchUpdateException; import java.sql.Connection; import java.sql.ResultSet; @@ -25,21 +28,24 @@ import java.sql.SQLWarning; import java.sql.Statement; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.List; import org.apache.ignite.cache.query.SqlQuery; import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode; -import org.apache.ignite.internal.processors.odbc.SqlStateCode; import org.apache.ignite.internal.processors.odbc.ClientListenerResponse; +import org.apache.ignite.internal.processors.odbc.SqlStateCode; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcBatchExecuteRequest; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcBatchExecuteResult; +import org.apache.ignite.internal.processors.odbc.jdbc.JdbcBulkLoadAckResult; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcQuery; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcQueryExecuteMultipleStatementsResult; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcQueryExecuteRequest; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcQueryExecuteResult; -import org.apache.ignite.internal.processors.odbc.jdbc.JdbcStatementType; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcResult; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcResultInfo; +import org.apache.ignite.internal.processors.odbc.jdbc.JdbcBulkLoadBatchRequest; +import org.apache.ignite.internal.processors.odbc.jdbc.JdbcStatementType; import static java.sql.ResultSet.CONCUR_READ_ONLY; import static java.sql.ResultSet.FETCH_FORWARD; @@ -132,6 +138,9 @@ protected void execute0(JdbcStatementType stmtType, String sql, List arg assert res0 != null; + if (res0 instanceof JdbcBulkLoadAckResult) + res0 = sendFile((JdbcBulkLoadAckResult)res0); + if (res0 instanceof JdbcQueryExecuteResult) { JdbcQueryExecuteResult res = (JdbcQueryExecuteResult)res0; @@ -176,6 +185,61 @@ else if (res0 instanceof JdbcQueryExecuteMultipleStatementsResult) { assert resultSets.size() > 0 : "At least one results set is expected"; } + /** + * Sends a file to server in batches via multiple {@link JdbcBulkLoadBatchRequest}s. + * + * @param cmdRes Result of invoking COPY command: contains server-parsed + * bulk load parameters, such as file name and batch size. + */ + private JdbcResult sendFile(JdbcBulkLoadAckResult cmdRes) throws SQLException { + String fileName = cmdRes.params().localFileName(); + int batchSize = cmdRes.params().batchSize(); + + int batchNum = 0; + + try { + try (InputStream input = new BufferedInputStream(new FileInputStream(fileName))) { + byte[] buf = new byte[batchSize]; + + int readBytes; + while ((readBytes = input.read(buf)) != -1) { + if (readBytes == 0) + continue; + + JdbcResult res = conn.sendRequest(new JdbcBulkLoadBatchRequest( + cmdRes.queryId(), + batchNum++, + JdbcBulkLoadBatchRequest.CMD_CONTINUE, + readBytes == buf.length ? buf : Arrays.copyOf(buf, readBytes))); + + if (!(res instanceof JdbcQueryExecuteResult)) + throw new SQLException("Unknown response sent by the server: " + res); + } + + return conn.sendRequest(new JdbcBulkLoadBatchRequest( + cmdRes.queryId(), + batchNum++, + JdbcBulkLoadBatchRequest.CMD_FINISHED_EOF)); + } + } + catch (Exception e) { + try { + conn.sendRequest(new JdbcBulkLoadBatchRequest( + cmdRes.queryId(), + batchNum, + JdbcBulkLoadBatchRequest.CMD_FINISHED_ERROR)); + } + catch (SQLException e1) { + throw new SQLException("Cannot send finalization request: " + e1.getMessage(), e); + } + + if (e instanceof SQLException) + throw (SQLException) e; + else + throw new SQLException("Failed to read file: '" + fileName + "'", SqlStateCode.INTERNAL_ERROR, e); + } + } + /** {@inheritDoc} */ @Override public int executeUpdate(String sql) throws SQLException { execute0(JdbcStatementType.UPDATE_STMT_TYPE, sql, null); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTask.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTask.java index aa9f009962db2..07034f4579e41 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTask.java @@ -32,6 +32,8 @@ import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteJdbcDriver; import org.apache.ignite.IgniteSystemProperties; +import org.apache.ignite.cache.query.BulkLoadContextCursor; +import org.apache.ignite.cache.query.FieldsQueryCursor; import org.apache.ignite.cache.query.QueryCursor; import org.apache.ignite.cache.query.SqlFieldsQuery; import org.apache.ignite.internal.IgniteKernal; @@ -168,7 +170,15 @@ public JdbcQueryTask(Ignite ignite, String cacheName, String schemaName, String qry.setLazy(lazy()); qry.setSchema(schemaName); - QueryCursorImpl> qryCursor = (QueryCursorImpl>)cache.withKeepBinary().query(qry); + FieldsQueryCursor> fldQryCursor = cache.withKeepBinary().query(qry); + + if (fldQryCursor instanceof BulkLoadContextCursor) { + fldQryCursor.close(); + + throw new SQLException("COPY command is currently supported only in thin JDBC driver."); + } + + QueryCursorImpl> qryCursor = (QueryCursorImpl>)fldQryCursor; if (isQry == null) isQry = qryCursor.isQuery(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadAckClientParameters.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadAckClientParameters.java new file mode 100644 index 0000000000000..119d9f94e9dcf --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadAckClientParameters.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.bulkload; + +import org.jetbrains.annotations.NotNull; + +/** + * Bulk load parameters, which are parsed from SQL command and sent from server to client. + */ +public class BulkLoadAckClientParameters { + /** Minimum batch size. */ + public static final int MIN_BATCH_SIZE = 1; + + /** + * Maximum batch size. Note that the batch is wrapped to transport objects and the overall packet should fit + * into a Java array. 512 has been chosen arbitrarily. + */ + public static final int MAX_BATCH_SIZE = Integer.MAX_VALUE - 512; + + /** Size of a file batch for COPY command. */ + public static final int DEFAULT_BATCH_SIZE = 4 * 1024 * 1024; + + /** Local name of the file to send to server */ + @NotNull private final String locFileName; + + /** File batch size in bytes. */ + private final int batchSize; + + /** + * Creates a bulk load parameters. + * + * @param locFileName File name to send from client to server. + * @param batchSize Batch size (Number of bytes in a portion of a file to send in one JDBC request/response). + */ + public BulkLoadAckClientParameters(@NotNull String locFileName, int batchSize) { + this.locFileName = locFileName; + this.batchSize = batchSize; + } + + /** + * Returns the local name of file to send. + * + * @return The local name of file to send. + */ + @NotNull public String localFileName() { + return locFileName; + } + + /** + * Returns the batch size. + * + * @return The batch size. + */ + public int batchSize() { + return batchSize; + } + + /** + * Checks if batch size value is valid. + * + * @param sz The batch size to check. + * @throws IllegalArgumentException if batch size is invalid. + */ + public static boolean isValidBatchSize(int sz) { + return sz >= MIN_BATCH_SIZE && sz <= MAX_BATCH_SIZE; + } + + /** + * Creates proper batch size error message if {@link #isValidBatchSize(int)} check has failed. + * + * @param sz The batch size. + * @return The string with the error message. + */ + public static String batchSizeErrorMsg(int sz) { + return "Batch size should be within [" + MIN_BATCH_SIZE + ".." + MAX_BATCH_SIZE + "]: " + sz; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadCacheWriter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadCacheWriter.java new file mode 100644 index 0000000000000..90714c8308de4 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadCacheWriter.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.bulkload; + +import org.apache.ignite.lang.IgniteBiTuple; +import org.apache.ignite.lang.IgniteInClosure; + +/** A proxy, which stores given key+value pair to a cache. */ +public abstract class BulkLoadCacheWriter implements IgniteInClosure>, AutoCloseable { + /** + * Returns number of entry updates made by the writer. + * + * @return The number of cache entry updates. + */ + public abstract long updateCnt(); +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadCsvFormat.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadCsvFormat.java new file mode 100644 index 0000000000000..6f5e91e8194fe --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadCsvFormat.java @@ -0,0 +1,159 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.bulkload; + +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.util.regex.Pattern; + +/** A placeholder for bulk load CSV format parser options. */ +public class BulkLoadCsvFormat extends BulkLoadFormat { + + /** Line separator pattern. */ + @NotNull public static final Pattern DEFAULT_LINE_SEPARATOR = Pattern.compile("[\r\n]+"); + + /** Field separator pattern. */ + @NotNull public static final Pattern DEFAULT_FIELD_SEPARATOR = Pattern.compile(","); + + /** Quote characters */ + @NotNull public static final String DEFAULT_QUOTE_CHARS = "\""; + + /** Default escape sequence start characters. */ + @Nullable public static final String DEFAULT_ESCAPE_CHARS = null; + + /** Line comment start pattern. */ + @Nullable public static final Pattern DEFAULT_COMMENT_CHARS = null; + + /** Format name. */ + public static final String NAME = "CSV"; + + /** Line separator pattern. */ + @Nullable private Pattern lineSeparator; + + /** Field separator pattern. */ + @Nullable private Pattern fieldSeparator; + + /** Set of quote characters. */ + @Nullable private String quoteChars; + + /** Line comment start pattern. */ + @Nullable private Pattern commentChars; + + /** Set of escape start characters. */ + @Nullable private String escapeChars; + + /** + * Returns the name of the format. + * + * @return The name of the format. + */ + @Override public String name() { + return NAME; + } + + /** + * Returns the line separator pattern. + * + * @return The line separator pattern. + */ + @Nullable public Pattern lineSeparator() { + return lineSeparator; + } + + /** + * Sets the line separator pattern. + * + * @param lineSeparator The line separator pattern. + */ + public void lineSeparator(@Nullable Pattern lineSeparator) { + this.lineSeparator = lineSeparator; + } + + /** + * Returns the field separator pattern. + * + * @return The field separator pattern. + */ + @Nullable public Pattern fieldSeparator() { + return fieldSeparator; + } + + /** + * Sets the field separator pattern. + * + * @param fieldSeparator The field separator pattern. + */ + public void fieldSeparator(@Nullable Pattern fieldSeparator) { + this.fieldSeparator = fieldSeparator; + } + + /** + * Returns the quote characters. + * + * @return The quote characters. + */ + @Nullable public String quoteChars() { + return quoteChars; + } + + /** + * Sets the quote characters. + * + * @param quoteChars The quote characters. + */ + public void quoteChars(@Nullable String quoteChars) { + this.quoteChars = quoteChars; + } + + /** + * Returns the line comment start pattern. + * + * @return The line comment start pattern. + */ + @Nullable public Pattern commentChars() { + return commentChars; + } + + /** + * Sets the line comment start pattern. + * + * @param commentChars The line comment start pattern. + */ + public void commentChars(@Nullable Pattern commentChars) { + this.commentChars = commentChars; + } + + /** + * Returns the escape characters. + * + * @return The escape characters. + */ + @Nullable public String escapeChars() { + return escapeChars; + } + + /** + * Sets the escape characters. + * + * @param escapeChars The escape characters. + */ + public void escapeChars(@Nullable String escapeChars) { + this.escapeChars = escapeChars; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadCsvParser.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadCsvParser.java new file mode 100644 index 0000000000000..0511596012477 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadCsvParser.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.bulkload; + +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.processors.bulkload.pipeline.CharsetDecoderBlock; +import org.apache.ignite.internal.processors.bulkload.pipeline.CsvLineProcessorBlock; +import org.apache.ignite.internal.processors.bulkload.pipeline.PipelineBlock; +import org.apache.ignite.internal.processors.bulkload.pipeline.StrListAppenderBlock; +import org.apache.ignite.internal.processors.bulkload.pipeline.LineSplitterBlock; + +import java.util.LinkedList; +import java.util.List; + +/** CSV parser for COPY command. */ +public class BulkLoadCsvParser extends BulkLoadParser { + /** Processing pipeline input block: a decoder for the input stream of bytes */ + private final PipelineBlock inputBlock; + + /** A record collecting block that appends its input to {@code List}. */ + private final StrListAppenderBlock collectorBlock; + + /** + * Creates bulk load CSV parser. + * + * @param format Format options (parsed from COPY command). + */ + public BulkLoadCsvParser(BulkLoadCsvFormat format) { + inputBlock = new CharsetDecoderBlock(BulkLoadFormat.DEFAULT_INPUT_CHARSET); + + collectorBlock = new StrListAppenderBlock(); + + // Handling of the other options is to be implemented in IGNITE-7537. + inputBlock.append(new LineSplitterBlock(format.lineSeparator())) + .append(new CsvLineProcessorBlock(format.fieldSeparator(), format.quoteChars())) + .append(collectorBlock); + } + + /** {@inheritDoc} */ + @Override protected Iterable> parseBatch(byte[] batchData, boolean isLastBatch) + throws IgniteCheckedException { + List> res = new LinkedList<>(); + + collectorBlock.output(res); + + inputBlock.accept(batchData, isLastBatch); + + return res; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadFormat.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadFormat.java new file mode 100644 index 0000000000000..cff93c5788552 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadFormat.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.bulkload; + +import java.nio.charset.Charset; + +/** A superclass and a factory for bulk load format options. */ +public abstract class BulkLoadFormat { + /** The default input charset. */ + public static final Charset DEFAULT_INPUT_CHARSET = Charset.forName("UTF-8"); + + /** + * Returns the format name. + * + * @return The format name. + */ + public abstract String name(); +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadParser.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadParser.java new file mode 100644 index 0000000000000..252e87b1a9bc6 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadParser.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.bulkload; + +import org.apache.ignite.IgniteCheckedException; + +import java.util.List; + +/** + * Bulk load file format parser superclass + factory of known formats. + * + *

The parser processes a batch of input data and return a list of records. + * + *

The parser uses corresponding options from {@link BulkLoadFormat} subclass. + */ +public abstract class BulkLoadParser { + /** + * Parses a batch of input data and returns a list of records parsed + * (in most cases this is a list of strings). + * + *

Note that conversion between parsed and database table type is done by the other + * object (see {@link BulkLoadProcessor#dataConverter}) by the request processing code. + * This method is not obliged to do this conversion. + * + * @param batchData Data from the current batch. + * @param isLastBatch true if this is the last batch. + * @return The list of records. + * @throws IgniteCheckedException If any processing error occurs. + */ + protected abstract Iterable> parseBatch(byte[] batchData, boolean isLastBatch) + throws IgniteCheckedException; + + /** + * Creates a parser for a given format options. + * + * @param format The input format object. + * @return The parser. + * @throws IllegalArgumentException if the format is not known to the factory. + */ + public static BulkLoadParser createParser(BulkLoadFormat format) { + if (format instanceof BulkLoadCsvFormat) + return new BulkLoadCsvParser((BulkLoadCsvFormat)format); + + throw new IllegalArgumentException("Internal error: format is not defined"); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadProcessor.java new file mode 100644 index 0000000000000..ccf3e251e539e --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadProcessor.java @@ -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. + */ + +package org.apache.ignite.internal.processors.bulkload; + +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteDataStreamer; +import org.apache.ignite.IgniteIllegalStateException; +import org.apache.ignite.internal.util.lang.IgniteClosureX; +import org.apache.ignite.lang.IgniteBiTuple; + +import java.util.List; + +/** + * Bulk load (COPY) command processor used on server to keep various context data and process portions of input + * received from the client side. + */ +public class BulkLoadProcessor implements AutoCloseable { + /** Parser of the input bytes. */ + private final BulkLoadParser inputParser; + + /** + * Converter, which transforms the list of strings parsed from the input stream to the key+value entry to add to + * the cache. + */ + private final IgniteClosureX, IgniteBiTuple> dataConverter; + + /** Streamer that puts actual key/value into the cache. */ + private final BulkLoadCacheWriter outputStreamer; + + /** Becomes true after {@link #close()} method is called. */ + private boolean isClosed; + + /** + * Creates bulk load processor. + * + * @param inputParser Parser of the input bytes. + * @param dataConverter Converter, which transforms the list of strings parsed from the input stream to the + * key+value entry to add to the cache. + * @param outputStreamer Streamer that puts actual key/value into the cache. + */ + public BulkLoadProcessor(BulkLoadParser inputParser, IgniteClosureX, IgniteBiTuple> dataConverter, + BulkLoadCacheWriter outputStreamer) { + this.inputParser = inputParser; + this.dataConverter = dataConverter; + this.outputStreamer = outputStreamer; + isClosed = false; + } + + /** + * Returns the streamer that puts actual key/value into the cache. + * + * @return Streamer that puts actual key/value into the cache. + */ + public BulkLoadCacheWriter outputStreamer() { + return outputStreamer; + } + + /** + * Processes the incoming batch and writes data to the cache by calling the data converter and output streamer. + * + * @param batchData Data from the current batch. + * @param isLastBatch true if this is the last batch. + * @throws IgniteIllegalStateException when called after {@link #close()}. + */ + public void processBatch(byte[] batchData, boolean isLastBatch) throws IgniteCheckedException { + if (isClosed) + throw new IgniteIllegalStateException("Attempt to process a batch on a closed BulkLoadProcessor"); + + Iterable> inputRecords = inputParser.parseBatch(batchData, isLastBatch); + + for (List record : inputRecords) { + IgniteBiTuple kv = dataConverter.apply(record); + + outputStreamer.apply(kv); + } + } + + /** + * Aborts processing and closes the underlying objects ({@link IgniteDataStreamer}). + */ + @Override public void close() throws Exception { + if (isClosed) + return; + + isClosed = true; + + outputStreamer.close(); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadStreamerWriter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadStreamerWriter.java new file mode 100644 index 0000000000000..3e5efd941152e --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadStreamerWriter.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.bulkload; + +import org.apache.ignite.IgniteDataStreamer; +import org.apache.ignite.lang.IgniteBiTuple; + +/** + * A bulk load cache writer object that adds entries using {@link IgniteDataStreamer}. + */ +public class BulkLoadStreamerWriter extends BulkLoadCacheWriter { + /** Serialization version UID. */ + private static final long serialVersionUID = 0L; + + /** The streamer. */ + private final IgniteDataStreamer streamer; + + /** + * A number of {@link IgniteDataStreamer#addData(Object, Object)} calls made, + * since we don't have any kind of result data back from the streamer. + */ + private long updateCnt; + + /** + * Creates a cache writer. + * + * @param streamer The streamer to use. + */ + public BulkLoadStreamerWriter(IgniteDataStreamer streamer) { + this.streamer = streamer; + updateCnt = 0; + } + + /** {@inheritDoc} */ + @Override public void apply(IgniteBiTuple entry) { + streamer.addData(entry.getKey(), entry.getValue()); + + updateCnt++; + } + + /** {@inheritDoc} */ + @Override public void close() { + streamer.close(); + } + + /** {@inheritDoc} */ + @Override public long updateCnt() { + return updateCnt; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/pipeline/CharsetDecoderBlock.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/pipeline/CharsetDecoderBlock.java new file mode 100644 index 0000000000000..5b18def1a37cc --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/pipeline/CharsetDecoderBlock.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.bulkload.pipeline; + +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteIllegalStateException; + +import java.nio.ByteBuffer; +import java.nio.CharBuffer; +import java.nio.charset.Charset; +import java.nio.charset.CharsetDecoder; +import java.nio.charset.CoderResult; +import java.nio.charset.CodingErrorAction; +import java.util.Arrays; + +/** + * A {@link PipelineBlock}, which converts stream of bytes supplied as byte[] arrays to an array of char[] using + * the specified encoding. Decoding errors (malformed input and unmappable characters) are to handled by dropping + * the erroneous input, appending the coder's replacement value to the output buffer, and resuming the coding operation. + */ +public class CharsetDecoderBlock extends PipelineBlock { + /** Charset decoder */ + private final CharsetDecoder charsetDecoder; + + /** Leftover bytes (partial characters) from the last batch, + * or null if everything was processed. */ + private byte[] leftover; + + /** True once we've reached the end of input. */ + private boolean isEndOfInput; + + /** + * Creates charset decoder block. + * + * @param charset The charset encoding to decode bytes from. + */ + public CharsetDecoderBlock(Charset charset) { + charsetDecoder = charset.newDecoder() + .onMalformedInput(CodingErrorAction.REPLACE) + .onUnmappableCharacter(CodingErrorAction.REPLACE); + + isEndOfInput = false; + leftover = null; + } + + /** {@inheritDoc} */ + @Override public void accept(byte[] data, boolean isLastAppend) throws IgniteCheckedException { + assert nextBlock != null; + + assert !isEndOfInput : "convertBytes() called after end of input"; + + isEndOfInput = isLastAppend; + + if (leftover == null && data.length == 0) { + nextBlock.accept(new char[0], isLastAppend); + return; + } + + ByteBuffer dataBuf; + + if (leftover == null) + dataBuf = ByteBuffer.wrap(data); + else { + dataBuf = ByteBuffer.allocate(leftover.length + data.length); + + dataBuf.put(leftover) + .put(data); + + dataBuf.flip(); + + leftover = null; + } + + int outBufLen = (int)Math.ceil(charsetDecoder.maxCharsPerByte() * (data.length + 1)); + + assert outBufLen > 0; + + CharBuffer outBuf = CharBuffer.allocate(outBufLen); + + for (;;) { + CoderResult res = charsetDecoder.decode(dataBuf, outBuf, isEndOfInput); + + if (res.isUnderflow()) { + // End of input buffer reached. Either skip the partial character at the end or wait for the next batch. + if (!isEndOfInput && dataBuf.remaining() > 0) + leftover = Arrays.copyOfRange(dataBuf.array(), + dataBuf.arrayOffset() + dataBuf.position(), dataBuf.limit()); + + if (isEndOfInput) + charsetDecoder.flush(outBuf); // See {@link CharsetDecoder} class javadoc for the protocol. + + if (outBuf.position() > 0) + nextBlock.accept(Arrays.copyOfRange(outBuf.array(), outBuf.arrayOffset(), outBuf.position()), + isEndOfInput); + + break; + } + + if (res.isOverflow()) { // Not enough space in the output buffer, flush it and retry. + assert outBuf.position() > 0; + + nextBlock.accept(Arrays.copyOfRange(outBuf.array(), outBuf.arrayOffset(), outBuf.position()), + isEndOfInput); + + outBuf.flip(); + + continue; + } + + assert ! res.isMalformed() && ! res.isUnmappable(); + + // We're not supposed to reach this point with the current implementation. + // The code below will fire exception if Oracle implementation of CharsetDecoder will be changed in future. + throw new IgniteIllegalStateException("Unknown CharsetDecoder state"); + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/pipeline/CsvLineProcessorBlock.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/pipeline/CsvLineProcessorBlock.java new file mode 100644 index 0000000000000..5b2ee4b9f3ced --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/pipeline/CsvLineProcessorBlock.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.processors.bulkload.pipeline; + +import org.apache.ignite.IgniteCheckedException; +import org.jetbrains.annotations.NotNull; + +import java.util.regex.Pattern; + +/** + * A {@link PipelineBlock}, which splits line according to CSV format rules and unquotes fields. + * The next block {@link PipelineBlock#accept(Object, boolean)} is called per-line. + */ +public class CsvLineProcessorBlock extends PipelineBlock { + /** Field delimiter pattern. */ + private final Pattern fldDelim; + + /** Quote character. */ + private final String quoteChars; + + /** + * Creates a CSV line parser. + * + * @param fldDelim The pattern for the field delimiter. + * @param quoteChars Quoting character. + */ + public CsvLineProcessorBlock(Pattern fldDelim, String quoteChars) { + this.fldDelim = fldDelim; + this.quoteChars = quoteChars; + } + + /** {@inheritDoc} */ + @Override public void accept(String input, boolean isLastPortion) throws IgniteCheckedException { + // Currently we don't process quoted field delimiter properly, will be fixed in IGNITE-7537. + String[] fields = fldDelim.split(input); + + for (int i = 0; i < fields.length; i++) + fields[i] = trim(fields[i]); + + nextBlock.accept(fields, isLastPortion); + } + + /** + * Trims quote characters from beginning and end of the line. + * + * @param str String to trim. + * @return The trimmed string. + */ + @NotNull private String trim(String str) { + int startPos = quoteChars.indexOf(str.charAt(0)) != -1 ? 1 : 0; + int endPos = quoteChars.indexOf(str.charAt(str.length() - 1)) != -1 ? str.length() - 1 : str.length(); + + return str.substring(startPos, endPos); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/pipeline/LineSplitterBlock.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/pipeline/LineSplitterBlock.java new file mode 100644 index 0000000000000..122d0db4ca7b3 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/pipeline/LineSplitterBlock.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.processors.bulkload.pipeline; + +import org.apache.ignite.IgniteCheckedException; + +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * A {@link PipelineBlock}, which splits input stream of char[] into lines using the specified {@link Pattern} + * as line separator. Next block {@link PipelineBlock#accept(Object, boolean)} is invoked for each line. + * Leftover characters are remembered and used during processing the next input batch, + * unless isLastPortion flag is specified. + */ +public class LineSplitterBlock extends PipelineBlock { + /** Line separator pattern */ + private final Pattern delim; + + /** Leftover characters from the previous invocation of {@link #accept(char[], boolean)}. */ + private StringBuilder leftover = new StringBuilder(); + + /** + * Creates line splitter block. + * + * @param delim The line separator pattern. + */ + public LineSplitterBlock(Pattern delim) { + this.delim = delim; + } + + /** {@inheritDoc} */ + @Override public void accept(char[] chars, boolean isLastPortion) throws IgniteCheckedException { + leftover.append(chars); + + String input = leftover.toString(); + Matcher matcher = delim.matcher(input); + + int lastPos = 0; + while (matcher.find()) { + String outStr = input.substring(lastPos, matcher.start()); + + if (!outStr.isEmpty()) + nextBlock.accept(outStr, false); + + lastPos = matcher.end(); + } + + if (lastPos != 0) + leftover.delete(0, lastPos); + + if (isLastPortion && leftover.length() > 0) { + nextBlock.accept(leftover.toString(), true); + leftover.setLength(0); + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/pipeline/PipelineBlock.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/pipeline/PipelineBlock.java new file mode 100644 index 0000000000000..914b4b4d4aaa4 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/pipeline/PipelineBlock.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.bulkload.pipeline; + +import org.apache.ignite.IgniteCheckedException; +import org.jetbrains.annotations.Nullable; + +/** + * A file parsing pipeline block. Accepts an portion of an input (isLastPortion flag is provided to signify the last + * block to process) and optionally calls the next block with transformed input or performs any other handling, + * such as storing input to internal structures. + */ +public abstract class PipelineBlock { + /** The next block in pipeline or null if this block is a terminator. */ + @Nullable protected PipelineBlock nextBlock; + + /** + * Creates a pipeline block. + * + *

(There is no nextBlock argument in the constructor: setting the next block using + * {@link #append(PipelineBlock)} method is more convenient. + */ + protected PipelineBlock() { + nextBlock = null; + } + + /** + * Sets the next block in this block and returns the next block. + * + *

Below is an example of using this method to set up a pipeline:
+ * {@code block1.append(block2).append(block3); }. + *

Block2 here becomes the next for block1, and block3 is the next one for the block2. + * + * @param next The next block for the current block. + * @return The next block ({@code next} argument). + */ + public PipelineBlock append(PipelineBlock next) { + nextBlock = next; + return next; + } + + /** + * Accepts a portion of input. {@code isLastPortion} parameter should be set if this is a last portion + * of the input. The method must not be called after the end of input: the call with {@code isLastPortion == true} + * is the last one. + * + * @param inputPortion Portion of input. + * @param isLastPortion Is this the last portion. + */ + public abstract void accept(I inputPortion, boolean isLastPortion) throws IgniteCheckedException; +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/pipeline/StrListAppenderBlock.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/pipeline/StrListAppenderBlock.java new file mode 100644 index 0000000000000..91cbc1e11117d --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/pipeline/StrListAppenderBlock.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.bulkload.pipeline; + +import java.util.Arrays; +import java.util.List; + +/** + * The PipelineBlock which appends its input to a user-supplied list. + * + *

The list is set using {@link #output(List)} method. + */ +public class StrListAppenderBlock extends PipelineBlock { + /** The output list. */ + private List> output; + + /** + * Creates the block. List can be configured using {@link #output(List)} method. + */ + public StrListAppenderBlock() { + output = null; + } + + /** + * Sets the output list. + * + * @param output The output list. + */ + public void output(List> output) { + this.output = output; + } + + /** {@inheritDoc} */ + @Override public void accept(String[] elements, boolean isLastPortion) { + output.add(Arrays.asList(elements)); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBulkLoadAckResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBulkLoadAckResult.java new file mode 100644 index 0000000000000..8a170ab64b8c7 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBulkLoadAckResult.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.internal.processors.odbc.jdbc; + +import org.apache.ignite.binary.BinaryObjectException; +import org.apache.ignite.internal.binary.BinaryReaderExImpl; +import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.processors.bulkload.BulkLoadAckClientParameters; +import org.apache.ignite.internal.sql.command.SqlBulkLoadCommand; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * A reply from server to SQL COPY command, which is essentially a request from server to client + * to send files from client to server (see IGNITE-6917 for details). + * + * @see JdbcBulkLoadProcessor for the protocol. + * @see SqlBulkLoadCommand + */ +public class JdbcBulkLoadAckResult extends JdbcResult { + /** Query ID for matching this command on server in further {@link JdbcBulkLoadBatchRequest} commands. */ + private long qryId; + + /** + * Bulk load parameters, which are parsed on the server side and sent to client to specify + * what files to send, batch size, etc. + */ + private BulkLoadAckClientParameters params; + + /**Creates uninitialized bulk load batch request result. */ + public JdbcBulkLoadAckResult() { + super(BULK_LOAD_ACK); + + qryId = 0; + params = null; + } + + /** + * Constructs a request from server (in form of reply) to send files from client to server. + * + * @param qryId Query ID to send in further {@link JdbcBulkLoadBatchRequest}s. + * @param params Various parameters for sending batches from client side. + */ + public JdbcBulkLoadAckResult(long qryId, BulkLoadAckClientParameters params) { + super(BULK_LOAD_ACK); + + this.qryId = qryId; + this.params = params; + } + + /** + * Returns the query ID. + * + * @return Query ID. + */ + public long queryId() { + return qryId; + } + + /** + * Returns the parameters for the client. + * + * @return The parameters for the client. + */ + public BulkLoadAckClientParameters params() { + return params; + } + + /** {@inheritDoc} */ + @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException { + super.writeBinary(writer); + + writer.writeLong(qryId); + writer.writeString(params.localFileName()); + writer.writeInt(params.batchSize()); + } + + /** {@inheritDoc} */ + @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException { + super.readBinary(reader); + + qryId = reader.readLong(); + + String locFileName = reader.readString(); + int batchSize = reader.readInt(); + + if (!BulkLoadAckClientParameters.isValidBatchSize(batchSize)) + throw new BinaryObjectException(BulkLoadAckClientParameters.batchSizeErrorMsg(batchSize)); + + params = new BulkLoadAckClientParameters(locFileName, batchSize); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(JdbcBulkLoadAckResult.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBulkLoadBatchRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBulkLoadBatchRequest.java new file mode 100644 index 0000000000000..b75de5a251c7c --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBulkLoadBatchRequest.java @@ -0,0 +1,183 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.jdbc; + +import org.apache.ignite.binary.BinaryObjectException; +import org.apache.ignite.internal.binary.BinaryReaderExImpl; +import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.sql.command.SqlBulkLoadCommand; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.jetbrains.annotations.NotNull; + +/** + * A JDBC request that sends a batch of a file to the server. Used when handling + * {@link SqlBulkLoadCommand} command. + */ +public class JdbcBulkLoadBatchRequest extends JdbcRequest { + /** A sentinel to indicate that {@link #cmd} field was not initialized. */ + public static final int CMD_UNKNOWN = -1; + + /** Next batch comes in this request and there are more batches. */ + public static final int CMD_CONTINUE = 0; + + /** + * This is the final batch from the client and there was an error on the client side, + * so terminate with error on the server side as well. + */ + public static final int CMD_FINISHED_ERROR = 1; + + /** + * This is the final batch of the file and everything went well on the client side. + * Server may complete the request. + */ + public static final int CMD_FINISHED_EOF = 2; + + /** QueryID of the original COPY command request. */ + private long qryId; + + /** Batch index starting from 0. */ + private int batchIdx; + + /** Command (see CMD_xxx constants above). */ + private int cmd; + + /** Data in this batch. */ + @NotNull private byte[] data; + + /** + * Creates the request with uninitialized parameters. + */ + public JdbcBulkLoadBatchRequest() { + super(BULK_LOAD_BATCH); + + qryId = -1; + batchIdx = -1; + cmd = CMD_UNKNOWN; + data = null; + } + + /** + * Creates the request with specified parameters and zero-length data. + * Typically used with {@link #CMD_FINISHED_ERROR} and {@link #CMD_FINISHED_EOF}. + * + * @param qryId The query ID from the {@link JdbcBulkLoadAckResult}. + * @param batchIdx Index of the current batch starting with 0. + * @param cmd The command ({@link #CMD_CONTINUE}, {@link #CMD_FINISHED_EOF}, or {@link #CMD_FINISHED_ERROR}). + */ + public JdbcBulkLoadBatchRequest(long qryId, int batchIdx, int cmd) { + this(qryId, batchIdx, cmd, new byte[0]); + } + + /** + * Creates the request with the specified parameters. + * + * @param qryId The query ID from the {@link JdbcBulkLoadAckResult}. + * @param batchIdx Index of the current batch starting with 0. + * @param cmd The command ({@link #CMD_CONTINUE}, {@link #CMD_FINISHED_EOF}, or {@link #CMD_FINISHED_ERROR}). + * @param data The data block (zero length is acceptable). + */ + public JdbcBulkLoadBatchRequest(long qryId, int batchIdx, int cmd, @NotNull byte[] data) { + super(BULK_LOAD_BATCH); + + this.qryId = qryId; + this.batchIdx = batchIdx; + + assert isCmdValid(cmd) : "Invalid command value: " + cmd; + this.cmd = cmd; + + this.data = data; + } + + /** + * Returns the original query ID. + * + * @return The original query ID. + */ + public long queryId() { + return qryId; + } + + /** + * Returns the batch index. + * + * @return The batch index. + */ + public long batchIdx() { + return batchIdx; + } + + /** + * Returns the command (see CMD_xxx constants for details). + * + * @return The command. + */ + public int cmd() { + return cmd; + } + + /** + * Returns the data. + * + * @return data if data was not supplied + */ + @NotNull public byte[] data() { + return data; + } + + /** {@inheritDoc} */ + @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException { + super.writeBinary(writer); + + writer.writeLong(qryId); + writer.writeInt(batchIdx); + writer.writeInt(cmd); + writer.writeByteArray(data); + } + + /** {@inheritDoc} */ + @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException { + super.readBinary(reader); + + qryId = reader.readLong(); + batchIdx = reader.readInt(); + + int c = reader.readInt(); + if (!isCmdValid(c)) + throw new BinaryObjectException("Invalid command: " + cmd); + + cmd = c; + + data = reader.readByteArray(); + assert data != null; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(JdbcBulkLoadBatchRequest.class, this); + } + + /** + * Checks if the command value is valid. + * + * @param c The command value to check. + * @return True if valid, false otherwise. + */ + private static boolean isCmdValid(int c) { + return c >= CMD_CONTINUE && c <= CMD_FINISHED_EOF; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBulkLoadProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBulkLoadProcessor.java new file mode 100644 index 0000000000000..97577917ff6c1 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBulkLoadProcessor.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.odbc.jdbc; + +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteIllegalStateException; +import org.apache.ignite.internal.processors.bulkload.BulkLoadProcessor; +import org.apache.ignite.internal.processors.query.IgniteSQLException; + +import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcBulkLoadBatchRequest.CMD_CONTINUE; +import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcBulkLoadBatchRequest.CMD_FINISHED_EOF; +import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcBulkLoadBatchRequest.CMD_FINISHED_ERROR; + +/** + * JDBC wrapper around {@link BulkLoadProcessor} that provides extra logic. + * + * Unlike other "single shot" request-reply commands, the + * COPY command the client-server interaction looks like this: + * + *

+ * Thin JDBC client                            Server
+ *        |                                       |
+ *        |------- JdbcQueryExecuteRequest ------>|
+ *        |         with SQL copy command         |
+ *        |                                       |
+ *        |<---- JdbcBulkLoadAckResult -----------|
+ *        | with BulkLoadAckClientParameters      |
+ *        | containing file name and batch size.  |
+ *        |                                       |
+ * (open the file,                                |
+ *  read portions and send them)                  |
+ *        |                                       |
+ *        |------- JdbcBulkLoadBatchRequest #1 -->|
+ *        | with a portion of input file.         |
+ *        |                                       |
+ *        |<--- JdbcQueryExecuteResult -----------|
+ *        | with current update counter.          |
+ *        |                                       |
+ *        |------- JdbcBulkLoadBatchRequest #2--->|
+ *        | with a portion of input file.         |
+ *        |                                       |
+ *        |<--- JdbcQueryExecuteResult -----------|
+ *        | with current update counter.          |
+ *        |                                       |
+ *        |------- JdbcBulkLoadBatchRequest #3--->|
+ *        | with the LAST portion of input file.  |
+ *        |                                       |
+ *        |<--- JdbcQueryExecuteResult -----------|
+ *        | with the final update counter.        |
+ *        |                                       |
+ * (close the file)                               |
+ *        |                                       |
+ * 
+ * + * In case of input file reading error, a flag is carried to the server: + * {@link JdbcBulkLoadBatchRequest#CMD_FINISHED_ERROR} and the processing + * is aborted on the both sides. + */ +public class JdbcBulkLoadProcessor { + /** A core processor that handles incoming data packets. */ + private final BulkLoadProcessor processor; + + /** Next batch index (for a very simple check that all batches were delivered to us). */ + protected long nextBatchIdx; + + /** + * Creates a JDBC-specific adapter for bulk load processor. + * + * @param processor Bulk load processor from the core to delegate calls to. + */ + public JdbcBulkLoadProcessor(BulkLoadProcessor processor) { + this.processor = processor; + nextBatchIdx = 0; + } + + /** + * Completely processes a bulk load batch request. + * + * Calls {@link BulkLoadProcessor} wrapping around some JDBC-specific logic + * (commands, bulk load batch index checking). + * + * @param req The current request. + */ + public void processBatch(JdbcBulkLoadBatchRequest req) + throws IgniteCheckedException { + if (nextBatchIdx != req.batchIdx()) + throw new IgniteSQLException("Batch #" + (nextBatchIdx + 1) + + " is missing. Received #" + req.batchIdx() + " instead."); + + nextBatchIdx++; + + switch (req.cmd()) { + case CMD_FINISHED_EOF: + processor.processBatch(req.data(), true); + + break; + + case CMD_CONTINUE: + processor.processBatch(req.data(), false); + + break; + + case CMD_FINISHED_ERROR: + break; + + default: + throw new IgniteIllegalStateException("Command was not recognized: " + req.cmd()); + } + } + + /** + * Closes the underlying objects. + * Currently we don't handle normal termination vs. abort. + */ + public void close() throws Exception { + processor.close(); + + nextBatchIdx = -1; + } + + /** + * Provides update counter for sending in the {@link JdbcBatchExecuteResult}. + * + * @return The update counter for sending in {@link JdbcBatchExecuteResult}. + */ + public long updateCnt() { + return processor.outputStreamer().updateCnt(); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequest.java index 385924cde5ac9..22522ad8b6ec2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequest.java @@ -60,6 +60,8 @@ public class JdbcRequest extends ClientListenerRequestNoId implements JdbcRawBin /** Get schemas metadata request. */ static final byte META_SCHEMAS = 12; + /** Send a batch of a data from client to server. */ + static final byte BULK_LOAD_BATCH = 13; /** Request type. */ private byte type; @@ -154,6 +156,11 @@ public static JdbcRequest readRequest(BinaryReaderExImpl reader) throws BinaryOb break; + case BULK_LOAD_BATCH: + req = new JdbcBulkLoadBatchRequest(); + + break; + default: throw new IgniteException("Unknown SQL listener request ID: [request ID=" + reqType + ']'); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java index 11b50ece95cc3..59fc06bcd02c7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java @@ -32,11 +32,14 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicLong; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.cache.query.BulkLoadContextCursor; import org.apache.ignite.cache.query.FieldsQueryCursor; import org.apache.ignite.cache.query.SqlFieldsQuery; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.IgniteVersionUtils; import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.processors.bulkload.BulkLoadAckClientParameters; +import org.apache.ignite.internal.processors.bulkload.BulkLoadProcessor; import org.apache.ignite.internal.processors.cache.QueryCursorImpl; import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode; import org.apache.ignite.internal.processors.cache.query.SqlFieldsQueryEx; @@ -57,9 +60,13 @@ import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteBiTuple; +import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcBulkLoadBatchRequest.CMD_CONTINUE; +import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcBulkLoadBatchRequest.CMD_FINISHED_EOF; +import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcBulkLoadBatchRequest.CMD_FINISHED_ERROR; import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcConnectionContext.VER_2_3_0; import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcConnectionContext.VER_2_4_0; import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.BATCH_EXEC; +import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.BULK_LOAD_BATCH; import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.META_COLUMNS; import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.META_INDEXES; import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.META_PARAMS; @@ -93,6 +100,9 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler { /** Current queries cursors. */ private final ConcurrentHashMap qryCursors = new ConcurrentHashMap<>(); + /** Current bulk load processors. */ + private final ConcurrentHashMap bulkLoadRequests = new ConcurrentHashMap<>(); + /** Distributed joins flag. */ private final boolean distributedJoins; @@ -197,6 +207,9 @@ public JdbcRequestHandler(GridKernalContext ctx, GridSpinBusyLock busyLock, int case META_SCHEMAS: return getSchemas((JdbcMetaSchemasRequest)req); + + case BULK_LOAD_BATCH: + return processBulkLoadFileBatch((JdbcBulkLoadBatchRequest)req); } return new JdbcResponse(IgniteQueryErrorCode.UNSUPPORTED_OPERATION, @@ -207,6 +220,46 @@ public JdbcRequestHandler(GridKernalContext ctx, GridSpinBusyLock busyLock, int } } + /** + * Processes a file batch sent from client as part of bulk load COPY command. + * + * @param req Request object with a batch of a file received from client. + * @return Response to send to the client. + */ + private ClientListenerResponse processBulkLoadFileBatch(JdbcBulkLoadBatchRequest req) { + JdbcBulkLoadProcessor processor = bulkLoadRequests.get(req.queryId()); + + if (ctx == null) + return new JdbcResponse(IgniteQueryErrorCode.UNEXPECTED_OPERATION, "Unknown query ID: " + + req.queryId() + ". Bulk load session may have been reclaimed due to timeout."); + + try { + processor.processBatch(req); + + switch (req.cmd()) { + case CMD_FINISHED_ERROR: + case CMD_FINISHED_EOF: + bulkLoadRequests.remove(req.queryId()); + + processor.close(); + + break; + + case CMD_CONTINUE: + break; + + default: + throw new IllegalArgumentException(); + } + + return new JdbcResponse(new JdbcQueryExecuteResult(req.queryId(), processor.updateCnt())); + } + catch (Exception e) { + U.error(null, "Error processing file batch", e); + return new JdbcResponse(IgniteQueryErrorCode.UNKNOWN, "Server error: " + e); + } + } + /** {@inheritDoc} */ @Override public ClientListenerResponse handleException(Exception e, ClientListenerRequest req) { return exceptionToResult(e); @@ -237,6 +290,17 @@ public void onDisconnect() { { for (JdbcQueryCursor cursor : qryCursors.values()) cursor.close(); + + for (JdbcBulkLoadProcessor processor : bulkLoadRequests.values()) { + try { + processor.close(); + } + catch (Exception e) { + U.error(null, "Error closing JDBC bulk load processor.", e); + } + } + + bulkLoadRequests.clear(); } finally { busyLock.leaveBusy(); @@ -310,10 +374,22 @@ private JdbcResponse executeQuery(JdbcQueryExecuteRequest req) { List>> results = ctx.query().querySqlFields(qry, true, protocolVer.compareTo(VER_2_3_0) < 0); - if (results.size() == 1) { - FieldsQueryCursor> qryCur = results.get(0); + FieldsQueryCursor> fieldsCur = results.get(0); - JdbcQueryCursor cur = new JdbcQueryCursor(qryId, req.pageSize(), req.maxRows(), (QueryCursorImpl)qryCur); + if (fieldsCur instanceof BulkLoadContextCursor) { + BulkLoadContextCursor blCur = (BulkLoadContextCursor) fieldsCur; + + BulkLoadProcessor blProcessor = blCur.bulkLoadProcessor(); + BulkLoadAckClientParameters clientParams = blCur.clientParams(); + + bulkLoadRequests.put(qryId, new JdbcBulkLoadProcessor(blProcessor)); + + return new JdbcResponse(new JdbcBulkLoadAckResult(qryId, clientParams)); + } + + if (results.size() == 1) { + JdbcQueryCursor cur = new JdbcQueryCursor(qryId, req.pageSize(), req.maxRows(), + (QueryCursorImpl)fieldsCur); JdbcQueryExecuteResult res; @@ -350,8 +426,7 @@ private JdbcResponse executeQuery(JdbcQueryExecuteRequest req) { if (qryCur.isQuery()) { jdbcRes = new JdbcResultInfo(true, -1, qryId); - JdbcQueryCursor cur = new JdbcQueryCursor(qryId, req.pageSize(), req.maxRows(), - (QueryCursorImpl)qryCur); + JdbcQueryCursor cur = new JdbcQueryCursor(qryId, req.pageSize(), req.maxRows(), qryCur); qryCursors.put(qryId, cur); @@ -370,8 +445,6 @@ private JdbcResponse executeQuery(JdbcQueryExecuteRequest req) { return new JdbcResponse(new JdbcQueryExecuteMultipleStatementsResult(jdbcResults, items, last)); } - - } catch (Exception e) { qryCursors.remove(qryId); @@ -534,6 +607,9 @@ private void executeBatchedQuery(SqlFieldsQueryEx qry, List updCntsAcc, List>> qryRes = ctx.query().querySqlFields(qry, true, true); for (FieldsQueryCursor> cur : qryRes) { + if (cur instanceof BulkLoadContextCursor) + throw new IgniteSQLException("COPY command cannot be executed in batch mode."); + assert !((QueryCursorImpl)cur).isQuery(); Iterator> it = cur.iterator(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResult.java index 623a3391c4218..43631e96a6cbc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResult.java @@ -65,6 +65,9 @@ public class JdbcResult implements JdbcRawBinarylizable { /** Columns metadata result V3. */ static final byte META_COLUMNS_V3 = 15; + /** A request to send file from client to server. */ + static final byte BULK_LOAD_ACK = 16; + /** Success status. */ private byte type; @@ -163,6 +166,11 @@ public static JdbcResult readResult(BinaryReaderExImpl reader) throws BinaryObje break; + case BULK_LOAD_ACK: + res = new JdbcBulkLoadAckResult(); + + break; + default: throw new IgniteException("Unknown SQL listener request ID: [request ID=" + resId + ']'); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlKeyword.java b/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlKeyword.java index f5b8c3cedeb48..0238b0104d625 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlKeyword.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlKeyword.java @@ -42,6 +42,9 @@ public class SqlKeyword { /** Keyword: BOOL. */ public static final String BOOL = "BOOL"; + /** Keyword: BATCH_SIZE. */ + public static final String BATCH_SIZE = "BATCH_SIZE"; + /** Keyword: BOOLEAN. */ public static final String BOOLEAN = "BOOLEAN"; @@ -54,6 +57,9 @@ public class SqlKeyword { /** Keyword: CHARACTER. */ public static final String CHARACTER = "CHARACTER"; + /** Keyword: COPY. */ + public static final String COPY = "COPY"; + /** Keyword: CREATE. */ public static final String CREATE = "CREATE"; @@ -90,6 +96,12 @@ public class SqlKeyword { /** Keyword: FLOAT8. */ public static final String FLOAT8 = "FLOAT8"; + /** Keyword: FORMAT. */ + public static final String FORMAT = "FORMAT"; + + /** Keyword: FROM. */ + public static final String FROM = "FROM"; + /** Keyword: FULLTEXT. */ public static final String FULLTEXT = "FULLTEXT"; @@ -120,6 +132,9 @@ public class SqlKeyword { /** Keyword: INTEGER. */ public static final String INTEGER = "INTEGER"; + /** Keyword: INTO. */ + public static final String INTO = "INTO"; + /** Keyword: KEY. */ public static final String KEY = "KEY"; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlParser.java b/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlParser.java index 401ee98923c27..0627defee6ae5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlParser.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlParser.java @@ -18,12 +18,14 @@ package org.apache.ignite.internal.sql; import org.apache.ignite.internal.sql.command.SqlAlterTableCommand; +import org.apache.ignite.internal.sql.command.SqlBulkLoadCommand; import org.apache.ignite.internal.sql.command.SqlCommand; import org.apache.ignite.internal.sql.command.SqlCreateIndexCommand; import org.apache.ignite.internal.sql.command.SqlDropIndexCommand; import org.jetbrains.annotations.Nullable; import static org.apache.ignite.internal.sql.SqlKeyword.ALTER; +import static org.apache.ignite.internal.sql.SqlKeyword.COPY; import static org.apache.ignite.internal.sql.SqlKeyword.CREATE; import static org.apache.ignite.internal.sql.SqlKeyword.DROP; import static org.apache.ignite.internal.sql.SqlKeyword.HASH; @@ -103,6 +105,11 @@ private SqlCommand nextCommand0() { break; + case COPY: + cmd = processCopy(); + + break; + case ALTER: cmd = processAlter(); } @@ -115,7 +122,7 @@ private SqlCommand nextCommand0() { return cmd; } else - throw errorUnexpectedToken(lex, CREATE, DROP, ALTER); + throw errorUnexpectedToken(lex, CREATE, DROP, COPY, ALTER); case QUOTED: case MINUS: @@ -129,6 +136,15 @@ private SqlCommand nextCommand0() { } } + /** + * Processes COPY command. + * + * @return The {@link SqlBulkLoadCommand} command. + */ + private SqlCommand processCopy() { + return new SqlBulkLoadCommand().parse(lex); + } + /** * Process CREATE keyword. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/sql/command/SqlBulkLoadCommand.java b/modules/core/src/main/java/org/apache/ignite/internal/sql/command/SqlBulkLoadCommand.java new file mode 100644 index 0000000000000..e5246d5bd57b5 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/sql/command/SqlBulkLoadCommand.java @@ -0,0 +1,273 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.sql.command; + +import org.apache.ignite.internal.processors.bulkload.BulkLoadCsvFormat; +import org.apache.ignite.internal.processors.bulkload.BulkLoadFormat; +import org.apache.ignite.internal.processors.bulkload.BulkLoadAckClientParameters; +import org.apache.ignite.internal.sql.SqlKeyword; +import org.apache.ignite.internal.sql.SqlLexer; +import org.apache.ignite.internal.sql.SqlLexerTokenType; +import org.apache.ignite.internal.util.typedef.internal.S; + +import java.util.ArrayList; +import java.util.List; + +import static org.apache.ignite.internal.sql.SqlParserUtils.error; +import static org.apache.ignite.internal.sql.SqlParserUtils.parseIdentifier; +import static org.apache.ignite.internal.sql.SqlParserUtils.parseInt; +import static org.apache.ignite.internal.sql.SqlParserUtils.parseQualifiedIdentifier; +import static org.apache.ignite.internal.sql.SqlParserUtils.skipCommaOrRightParenthesis; +import static org.apache.ignite.internal.sql.SqlParserUtils.skipIfMatches; +import static org.apache.ignite.internal.sql.SqlParserUtils.skipIfMatchesKeyword; + +/** + * A parser for a COPY command (called 'bulk load' in the code, since word 'copy' is too generic). + */ +public class SqlBulkLoadCommand implements SqlCommand { + /** Local file name to send from client to server. */ + private String locFileName; + + /** Schema name + table name. */ + private SqlQualifiedName tblQName; + + /** User-specified list of columns. */ + private List cols; + + /** File format. */ + private BulkLoadFormat inputFormat; + + /** Batch size (size of portion of a file sent in each sub-request). */ + private Integer batchSize; + + /** + * Parses the command. + * + * @param lex The lexer. + * @return The parsed command object. + */ + @Override public SqlCommand parse(SqlLexer lex) { + skipIfMatchesKeyword(lex, SqlKeyword.FROM); // COPY keyword is already parsed + + parseFileName(lex); + + parseTableName(lex); + + parseColumns(lex); + + parseFormat(lex); + + parseParameters(lex); + + return this; + } + + /** + * Parses the file name. + * + * @param lex The lexer. + */ + private void parseFileName(SqlLexer lex) { + locFileName = parseIdentifier(lex); + } + + /** + * Parses the schema and table names. + * + * @param lex The lexer. + */ + private void parseTableName(SqlLexer lex) { + skipIfMatchesKeyword(lex, SqlKeyword.INTO); + + tblQName = parseQualifiedIdentifier(lex); + } + + /** + * Parses the list of columns. + * + * @param lex The lexer. + */ + private void parseColumns(SqlLexer lex) { + skipIfMatches(lex, SqlLexerTokenType.PARENTHESIS_LEFT); + + cols = new ArrayList<>(); + + do { + cols.add(parseColumn(lex)); + } + while (!skipCommaOrRightParenthesis(lex)); + } + + /** + * Parses column clause. + * + * @param lex The lexer. + * @return The column name. + */ + private String parseColumn(SqlLexer lex) { + return parseIdentifier(lex); + } + + /** + * Parses the format clause. + * + * @param lex The lexer. + */ + private void parseFormat(SqlLexer lex) { + skipIfMatchesKeyword(lex, SqlKeyword.FORMAT); + + String name = parseIdentifier(lex); + + switch (name.toUpperCase()) { + case BulkLoadCsvFormat.NAME: + BulkLoadCsvFormat fmt = new BulkLoadCsvFormat(); + + // IGNITE-7537 will introduce user-defined values + fmt.lineSeparator(BulkLoadCsvFormat.DEFAULT_LINE_SEPARATOR); + fmt.fieldSeparator(BulkLoadCsvFormat.DEFAULT_FIELD_SEPARATOR); + fmt.quoteChars(BulkLoadCsvFormat.DEFAULT_QUOTE_CHARS); + fmt.commentChars(BulkLoadCsvFormat.DEFAULT_COMMENT_CHARS); + fmt.escapeChars(BulkLoadCsvFormat.DEFAULT_ESCAPE_CHARS); + + inputFormat = fmt; + + break; + + default: + throw error(lex, "Unknown format name: " + name + + ". Currently supported format is " + BulkLoadCsvFormat.NAME); + } + } + + /** + * Parses the optional parameters. + * + * @param lex The lexer. + */ + private void parseParameters(SqlLexer lex) { + while (lex.lookAhead().tokenType() == SqlLexerTokenType.DEFAULT) { + switch (lex.lookAhead().token()) { + case SqlKeyword.BATCH_SIZE: + lex.shift(); + + int sz = parseInt(lex); + + if (!BulkLoadAckClientParameters.isValidBatchSize(sz)) + throw error(lex, BulkLoadAckClientParameters.batchSizeErrorMsg(sz)); + + batchSize = sz; + + break; + + default: + return; + } + } + } + + /** + * Returns the schemaName. + * + * @return schemaName. + */ + @Override public String schemaName() { + return tblQName.schemaName(); + } + + /** {@inheritDoc} */ + @Override public void schemaName(String schemaName) { + tblQName.schemaName(schemaName); + } + + /** + * Returns the table name. + * + * @return The table name + */ + public String tableName() { + return tblQName.name(); + } + + /** + * Sets the table name + * + * @param tblName The table name. + */ + public void tableName(String tblName) { + tblQName.name(tblName); + } + + /** + * Returns the local file name. + * + * @return The local file name. + */ + public String localFileName() { + return locFileName; + } + + /** + * Sets the local file name. + * + * @param locFileName The local file name. + */ + public void localFileName(String locFileName) { + this.locFileName = locFileName; + } + + /** + * Returns the list of columns. + * + * @return The list of columns. + */ + public List columns() { + return cols; + } + + /** + * Returns the input file format. + * + * @return The input file format. + */ + public BulkLoadFormat inputFormat() { + return inputFormat; + } + + /** + * Returns the batch size. + * + * @return The batch size. + */ + public Integer batchSize() { + return batchSize; + } + + /** + * Sets the batch size. + * + * @param batchSize The batch size. + */ + public void batchSize(int batchSize) { + this.batchSize = batchSize; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(SqlBulkLoadCommand.class, this); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/sql/SqlParserBulkLoadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/sql/SqlParserBulkLoadSelfTest.java new file mode 100644 index 0000000000000..b5cd55be2169f --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/sql/SqlParserBulkLoadSelfTest.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.sql; + +/** + * Tests for SQL parser: COPY command. + */ +public class SqlParserBulkLoadSelfTest extends SqlParserAbstractSelfTest { + /** + * Tests for COPY command. + * + * @throws Exception If any of sub-tests was failed. + */ + public void testCopy() { + assertParseError(null, + "copy grom \"any.file\" into Person (_key, age, firstName, lastName) format csv", + "Unexpected token: \"GROM\" (expected: \"FROM\")"); + + assertParseError(null, + "copy from into Person (_key, age, firstName, lastName) format csv", + "Unexpected token: \"INTO\" (expected: \"[identifier]\""); + + assertParseError(null, + "copy from any.file into Person (_key, age, firstName, lastName) format csv", + "Unexpected token: \".\" (expected: \"INTO\""); + + assertParseError(null, + "copy from \"any.file\" to Person (_key, age, firstName, lastName) format csv", + "Unexpected token: \"TO\" (expected: \"INTO\")"); + + // Column list + + assertParseError(null, + "copy from \"any.file\" into Person () format csv", + "Unexpected token: \")\" (expected: \"[identifier]\")"); + + assertParseError(null, + "copy from \"any.file\" into Person (,) format csv", + "Unexpected token: \",\" (expected: \"[identifier]\")"); + + assertParseError(null, + "copy from \"any.file\" into Person format csv", + "Unexpected token: \"FORMAT\" (expected: \"(\")"); + + // FORMAT + + assertParseError(null, + "copy from \"any.file\" into Person (_key, age, firstName, lastName)", + "Unexpected end of command (expected: \"FORMAT\")"); + + assertParseError(null, + "copy from \"any.file\" into Person (_key, age, firstName, lastName) format lsd", + "Unknown format name: LSD"); + } +} diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java index 72e80e2254d1a..c46c9062afe19 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java @@ -39,8 +39,15 @@ import org.apache.ignite.IgniteDataStreamer; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.cache.query.BulkLoadContextCursor; +import org.apache.ignite.cache.query.FieldsQueryCursor; import org.apache.ignite.cache.query.SqlFieldsQuery; import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.processors.bulkload.BulkLoadCacheWriter; +import org.apache.ignite.internal.processors.bulkload.BulkLoadProcessor; +import org.apache.ignite.internal.processors.bulkload.BulkLoadAckClientParameters; +import org.apache.ignite.internal.processors.bulkload.BulkLoadParser; +import org.apache.ignite.internal.processors.bulkload.BulkLoadStreamerWriter; import org.apache.ignite.internal.processors.cache.CacheOperationContext; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.QueryCursorImpl; @@ -58,8 +65,12 @@ import org.apache.ignite.internal.processors.query.h2.dml.UpdateMode; import org.apache.ignite.internal.processors.query.h2.dml.UpdatePlan; import org.apache.ignite.internal.processors.query.h2.dml.UpdatePlanBuilder; +import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table; import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQueryParser; +import org.apache.ignite.internal.sql.command.SqlBulkLoadCommand; +import org.apache.ignite.internal.sql.command.SqlCommand; import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashMap; +import org.apache.ignite.internal.util.lang.IgniteClosureX; import org.apache.ignite.internal.util.lang.IgniteSingletonIterator; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.T3; @@ -967,6 +978,67 @@ UpdateResult mapDistributedUpdate(String schemaName, PreparedStatement stmt, Sql return updateSqlFields(schemaName, c, GridSqlQueryParser.prepared(stmt), fldsQry, local, filter, cancel); } + /** + * Runs a DML statement for which we have internal command executor. + * + * @param sql The SQL command text to execute. + * @param cmd The command to execute. + * @return The cursor returned by the statement. + * @throws IgniteSQLException If failed. + */ + public FieldsQueryCursor> runNativeDmlStatement(String sql, SqlCommand cmd) { + try { + if (cmd instanceof SqlBulkLoadCommand) + return processBulkLoadCommand((SqlBulkLoadCommand)cmd); + else + throw new IgniteSQLException("Unsupported DML operation: " + sql, + IgniteQueryErrorCode.UNSUPPORTED_OPERATION); + + } + catch (IgniteSQLException e) { + throw e; + } + catch (Exception e) { + throw new IgniteSQLException("Unexpected DML operation failure: " + e.getMessage(), e); + } + } + + /** + * Process bulk load COPY command. + * + * @param cmd The command. + * @return The context (which is the result of the first request/response). + * @throws IgniteCheckedException If something failed. + */ + public FieldsQueryCursor> processBulkLoadCommand(SqlBulkLoadCommand cmd) throws IgniteCheckedException { + if (cmd.batchSize() == null) + cmd.batchSize(BulkLoadAckClientParameters.DEFAULT_BATCH_SIZE); + + GridH2Table tbl = idx.dataTable(cmd.schemaName(), cmd.tableName()); + + if (tbl == null) + throw new IgniteSQLException("Table does not exist: " + cmd.tableName(), + IgniteQueryErrorCode.TABLE_NOT_FOUND); + + UpdatePlan plan = UpdatePlanBuilder.planForBulkLoad(cmd, tbl); + + IgniteClosureX, IgniteBiTuple> dataConverter = new BulkLoadDataConverter(plan); + + GridCacheContext cache = tbl.cache(); + + IgniteDataStreamer streamer = cache.grid().dataStreamer(cache.name()); + + BulkLoadCacheWriter outputWriter = new BulkLoadStreamerWriter(streamer); + + BulkLoadParser inputParser = BulkLoadParser.createParser(cmd.inputFormat()); + + BulkLoadProcessor processor = new BulkLoadProcessor(inputParser, dataConverter, outputWriter); + + BulkLoadAckClientParameters params = new BulkLoadAckClientParameters(cmd.localFileName(), cmd.batchSize()); + + return new BulkLoadContextCursor(processor, params); + } + /** */ private final static class InsertEntryProcessor implements EntryProcessor { /** Value to set. */ @@ -1081,4 +1153,31 @@ static void checkUpdateResult(UpdateResult r) { } } + /** + * Converts a row of values to actual key+value using {@link UpdatePlan#processRow(List)}. + */ + private static class BulkLoadDataConverter extends IgniteClosureX, IgniteBiTuple> { + /** Update plan to convert incoming rows. */ + private final UpdatePlan plan; + + /** + * Creates the converter with the given update plan. + * + * @param plan The update plan to use. + */ + private BulkLoadDataConverter(UpdatePlan plan) { + this.plan = plan; + } + + /** + * Converts the record to a key+value. + * + * @param record The record to convert. + * @return The key+value. + * @throws IgniteCheckedException If conversion failed for some reason. + */ + @Override public IgniteBiTuple applyx(List record) throws IgniteCheckedException { + return plan.processRow(record); + } + } } 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 96b8935d87964..06c936b9867f6 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 @@ -43,6 +43,7 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; +import java.util.regex.Pattern; import javax.cache.Cache; import javax.cache.CacheException; import org.apache.ignite.IgniteCheckedException; @@ -99,7 +100,6 @@ import org.apache.ignite.internal.processors.query.h2.database.io.H2LeafIO; import org.apache.ignite.internal.processors.query.h2.ddl.DdlStatementsProcessor; import org.apache.ignite.internal.processors.query.h2.dml.DmlUtils; -import org.apache.ignite.internal.processors.query.h2.opt.DistributedJoinMode; import org.apache.ignite.internal.processors.query.h2.opt.GridH2DefaultTableEngine; import org.apache.ignite.internal.processors.query.h2.opt.GridH2IndexBase; import org.apache.ignite.internal.processors.query.h2.opt.GridH2PlainRowFactory; @@ -120,6 +120,7 @@ import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor; import org.apache.ignite.internal.sql.SqlParseException; import org.apache.ignite.internal.sql.SqlParser; +import org.apache.ignite.internal.sql.command.SqlBulkLoadCommand; import org.apache.ignite.internal.sql.command.SqlAlterTableCommand; import org.apache.ignite.internal.sql.command.SqlCommand; import org.apache.ignite.internal.sql.command.SqlCreateIndexCommand; @@ -190,6 +191,9 @@ */ @SuppressWarnings({"UnnecessaryFullyQualifiedName", "NonFinalStaticVariableUsedInClassInitialization"}) public class IgniteH2Indexing implements GridQueryIndexing { + public static final Pattern INTERNAL_CMD_RE = Pattern.compile( + "^(create|drop)\\s+index|^alter\\s+table|^copy", Pattern.CASE_INSENSITIVE); + /* * Register IO for indexes. */ @@ -1437,9 +1441,7 @@ UpdateResult runDistributedUpdate( */ private List>> tryQueryDistributedSqlFieldsNative(String schemaName, SqlFieldsQuery qry) { // Heuristic check for fast return. - String sqlUpper = qry.getSql().toUpperCase(); - - if (!(sqlUpper.contains("INDEX") || sqlUpper.contains("ALTER"))) + if (!INTERNAL_CMD_RE.matcher(qry.getSql().trim()).find()) return null; // Parse. @@ -1454,9 +1456,9 @@ private List>> tryQueryDistributedSqlFieldsNative(Stri if (parser.nextCommand() != null) return null; - // Only CREATE/DROP INDEX and ALTER TABLE commands are supported for now. + // Currently supported commands are: CREATE/DROP INDEX/COPY/ALTER TABLE if (!(cmd instanceof SqlCreateIndexCommand || cmd instanceof SqlDropIndexCommand || - cmd instanceof SqlAlterTableCommand)) + cmd instanceof SqlBulkLoadCommand || cmd instanceof SqlAlterTableCommand)) return null; } catch (Exception e) { @@ -1472,17 +1474,26 @@ private List>> tryQueryDistributedSqlFieldsNative(Stri if (e instanceof SqlParseException) code = ((SqlParseException)e).code(); - throw new IgniteSQLException("Failed to parse DDL statement: " + qry.getSql(), code, e); + throw new IgniteSQLException("Failed to parse DDL statement: " + qry.getSql() + ": " + e.getMessage(), + code, e); } // Execute. - try { - FieldsQueryCursor> res = ddlProc.runDdlStatement(qry.getSql(), cmd); + if (cmd instanceof SqlBulkLoadCommand) { + FieldsQueryCursor> cursor = dmlProc.runNativeDmlStatement(qry.getSql(), cmd); - return Collections.singletonList(res); + return Collections.singletonList(cursor); } - catch (IgniteCheckedException e) { - throw new IgniteSQLException("Failed to execute DDL statement [stmt=" + qry.getSql() + ']', e); + else { + try { + FieldsQueryCursor> cursor = ddlProc.runDdlStatement(qry.getSql(), cmd); + + return Collections.singletonList(cursor); + } + catch (IgniteCheckedException e) { + throw new IgniteSQLException("Failed to execute DDL statement [stmt=" + qry.getSql() + "]: " + + e.getMessage(), e); + } } } diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java index ca7680ae272ae..6f5b51f1a3651 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java @@ -62,6 +62,7 @@ import org.apache.ignite.internal.sql.command.SqlIndexColumn; import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.U; import org.h2.command.Prepared; import org.h2.command.ddl.AlterTableAlterColumn; import org.h2.command.ddl.CreateIndex; @@ -483,6 +484,7 @@ else if (stmt0 instanceof GridSqlAlterTableDropColumn) { return resCur; } catch (SchemaOperationException e) { + U.error(null, "DDL operation failure", e); throw convert(e); } catch (IgniteSQLException e) { diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdateMode.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdateMode.java index 0440648c193ec..d9c627a9dc9c0 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdateMode.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdateMode.java @@ -22,15 +22,18 @@ * or UPDATE/DELETE from subquery or literals/params based. */ public enum UpdateMode { - /** */ + /** MERGE command. */ MERGE, - /** */ + /** INSERT command. */ INSERT, - /** */ + /** UPDATE command. */ UPDATE, - /** */ + /** DELETE command. */ DELETE, + + /** COPY command. */ + BULK_LOAD } diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java index 17dc9d1a30b88..10d485a84c761 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java @@ -39,6 +39,7 @@ import org.h2.table.Column; import org.jetbrains.annotations.Nullable; +import static org.apache.ignite.internal.processors.query.h2.dml.UpdateMode.BULK_LOAD; import static org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOnheap.DEFAULT_COLUMNS_COUNT; /** @@ -182,6 +183,10 @@ public UpdatePlan( * @throws IgniteCheckedException if failed. */ public IgniteBiTuple processRow(List row) throws IgniteCheckedException { + if (mode != BULK_LOAD && row.size() != colNames.length) + throw new IgniteSQLException("Not enough values in a row: " + row.size() + " instead of " + colNames.length, + IgniteQueryErrorCode.ENTRY_PROCESSING); + GridH2RowDescriptor rowDesc = tbl.rowDescriptor(); GridQueryTypeDescriptor desc = rowDesc.type(); @@ -205,7 +210,8 @@ public UpdatePlan( if (key == null) { if (F.isEmpty(desc.keyFieldName())) - throw new IgniteSQLException("Key for INSERT or MERGE must not be null", IgniteQueryErrorCode.NULL_KEY); + throw new IgniteSQLException("Key for INSERT, COPY, or MERGE must not be null", + IgniteQueryErrorCode.NULL_KEY); else throw new IgniteSQLException("Null value is not allowed for column '" + desc.keyFieldName() + "'", IgniteQueryErrorCode.NULL_KEY); @@ -213,16 +219,18 @@ public UpdatePlan( if (val == null) { if (F.isEmpty(desc.valueFieldName())) - throw new IgniteSQLException("Value for INSERT, MERGE, or UPDATE must not be null", + throw new IgniteSQLException("Value for INSERT, COPY, MERGE, or UPDATE must not be null", IgniteQueryErrorCode.NULL_VALUE); else throw new IgniteSQLException("Null value is not allowed for column '" + desc.valueFieldName() + "'", IgniteQueryErrorCode.NULL_VALUE); } + int actualColCnt = Math.min(colNames.length, row.size()); + Map newColVals = new HashMap<>(); - for (int i = 0; i < colNames.length; i++) { + for (int i = 0; i < actualColCnt; i++) { if (i == keyColIdx || i == valColIdx) continue; @@ -241,14 +249,14 @@ public UpdatePlan( // We update columns in the order specified by the table for a reason - table's // column order preserves their precedence for correct update of nested properties. - Column[] cols = tbl.getColumns(); + Column[] tblCols = tbl.getColumns(); // First 3 columns are _key, _val and _ver. Skip 'em. - for (int i = DEFAULT_COLUMNS_COUNT; i < cols.length; i++) { + for (int i = DEFAULT_COLUMNS_COUNT; i < tblCols.length; i++) { if (tbl.rowDescriptor().isKeyValueOrVersionColumn(i)) continue; - String colName = cols[i].getName(); + String colName = tblCols[i].getName(); if (!newColVals.containsKey(colName)) continue; diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java index 3305b00f6a2dc..bced83667aff4 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java @@ -54,6 +54,7 @@ import org.apache.ignite.internal.processors.query.h2.sql.GridSqlTable; import org.apache.ignite.internal.processors.query.h2.sql.GridSqlUnion; import org.apache.ignite.internal.processors.query.h2.sql.GridSqlUpdate; +import org.apache.ignite.internal.sql.command.SqlBulkLoadCommand; import org.apache.ignite.internal.util.GridUnsafe; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.CU; @@ -406,6 +407,91 @@ else if (stmt instanceof GridSqlDelete) { } } + /** + * Prepare update plan for COPY command (AKA bulk load). + * + * @param cmd Bulk load command + * @return The update plan for this command. + * @throws IgniteCheckedException if failed. + */ + @SuppressWarnings("ConstantConditions") + public static UpdatePlan planForBulkLoad(SqlBulkLoadCommand cmd, GridH2Table tbl) throws IgniteCheckedException { + GridH2RowDescriptor desc = tbl.rowDescriptor(); + + if (desc == null) + throw new IgniteSQLException("Row descriptor undefined for table '" + tbl.getName() + "'", + IgniteQueryErrorCode.NULL_TABLE_DESCRIPTOR); + + GridCacheContext cctx = desc.context(); + + List cols = cmd.columns(); + + if (cols == null) + throw new IgniteSQLException("Columns are not defined", IgniteQueryErrorCode.NULL_TABLE_DESCRIPTOR); + + String[] colNames = new String[cols.size()]; + + int[] colTypes = new int[cols.size()]; + + int keyColIdx = -1; + int valColIdx = -1; + + boolean hasKeyProps = false; + boolean hasValProps = false; + + for (int i = 0; i < cols.size(); i++) { + String colName = cols.get(i); + + colNames[i] = colName; + + Column h2Col = tbl.getColumn(colName); + + colTypes[i] = h2Col.getType(); + int colId = h2Col.getColumnId(); + + if (desc.isKeyColumn(colId)) { + keyColIdx = i; + continue; + } + + if (desc.isValueColumn(colId)) { + valColIdx = i; + continue; + } + + GridQueryProperty prop = desc.type().property(colName); + + assert prop != null : "Property '" + colName + "' not found."; + + if (prop.key()) + hasKeyProps = true; + else + hasValProps = true; + } + + KeyValueSupplier keySupplier = createSupplier(cctx, desc.type(), keyColIdx, hasKeyProps, + true, false); + KeyValueSupplier valSupplier = createSupplier(cctx, desc.type(), valColIdx, hasValProps, + false, false); + + return new UpdatePlan( + UpdateMode.BULK_LOAD, + tbl, + colNames, + colTypes, + keySupplier, + valSupplier, + keyColIdx, + valColIdx, + null, + true, + null, + 0, + null, + null + ); + } + /** * Detect appropriate method of instantiating key or value (take from param, create binary builder, * invoke default ctor, or allocate). 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 68610a1493282..6295d8d3525c7 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 @@ -165,6 +165,7 @@ import org.apache.ignite.internal.processors.query.h2.sql.H2CompareBigQueryDistributedJoinsTest; import org.apache.ignite.internal.processors.query.h2.sql.H2CompareBigQueryTest; import org.apache.ignite.internal.processors.sql.SqlConnectorConfigurationValidationSelfTest; +import org.apache.ignite.internal.sql.SqlParserBulkLoadSelfTest; import org.apache.ignite.internal.sql.SqlParserCreateIndexSelfTest; import org.apache.ignite.internal.sql.SqlParserDropIndexSelfTest; import org.apache.ignite.spi.communication.tcp.GridOrderedMessageCancelSelfTest; @@ -183,6 +184,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(SqlParserCreateIndexSelfTest.class); suite.addTestSuite(SqlParserDropIndexSelfTest.class); + suite.addTestSuite(SqlParserBulkLoadSelfTest.class); suite.addTestSuite(SqlConnectorConfigurationValidationSelfTest.class); suite.addTestSuite(ClientConnectorConfigurationValidationSelfTest.class); diff --git a/parent/pom.xml b/parent/pom.xml index a958b5af658ea..8b370110d1001 100644 --- a/parent/pom.xml +++ b/parent/pom.xml @@ -810,6 +810,7 @@ **/*index*.md **/*.timestamp **/*.iml + **/*.csv **/pom-installed.xml **/keystore/*.jks **/keystore/*.pem @@ -837,11 +838,9 @@ **/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 **/resources/datasets/knn/* **/resources/datasets/regression/* **/ml/knn/datasets/* - examples/src/main/resources/person.csv src/main/java/org/jetbrains/annotations/*.java dev-tools/IGNITE-*.patch dev-tools/.gradle/**/* From c5e386ca96750213bddcd98d0af0c589fee476ca Mon Sep 17 00:00:00 2001 From: gg-shq Date: Wed, 7 Feb 2018 18:31:27 +0300 Subject: [PATCH 019/314] IGNITE-7586: Added COPY command into the JDBC example. This closes #3485 --- .../examples/sql/SqlJdbcCopyExample.java | 107 ++++++++++++++++++ examples/src/main/resources/cityBulkLoad.csv | 3 + .../src/main/resources/personBulkLoad.csv | 4 + 3 files changed, 114 insertions(+) create mode 100644 examples/src/main/java/org/apache/ignite/examples/sql/SqlJdbcCopyExample.java create mode 100644 examples/src/main/resources/cityBulkLoad.csv create mode 100644 examples/src/main/resources/personBulkLoad.csv diff --git a/examples/src/main/java/org/apache/ignite/examples/sql/SqlJdbcCopyExample.java b/examples/src/main/java/org/apache/ignite/examples/sql/SqlJdbcCopyExample.java new file mode 100644 index 0000000000000..9ac6419d2d56a --- /dev/null +++ b/examples/src/main/java/org/apache/ignite/examples/sql/SqlJdbcCopyExample.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.examples.sql; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.Statement; +import org.apache.ignite.examples.ExampleNodeStartup; +import org.apache.ignite.internal.util.IgniteUtils; + +/** + * This example demonstrates usage of COPY command via Ignite thin JDBC driver. + *

+ * Ignite nodes must be started in separate process using {@link ExampleNodeStartup} before running this example. + */ +public class SqlJdbcCopyExample { + /** + * Executes JDBC COPY example. + * + * @param args Command line arguments, none required. + * @throws Exception If example execution failed. + */ + public static void main(String[] args) throws Exception { + print("JDBC COPY example started."); + + // Open JDBC connection + try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1/")) { + print("Connected to server."); + + // Create database objects. + try (Statement stmt = conn.createStatement()) { + // Create reference City table based on REPLICATED template. + stmt.executeUpdate("CREATE TABLE city (id LONG PRIMARY KEY, name VARCHAR) " + + "WITH \"template=replicated\""); + + // Create table based on PARTITIONED template with one backup. + stmt.executeUpdate("CREATE TABLE person (id LONG, name VARCHAR, city_id LONG, " + + "PRIMARY KEY (id, city_id)) WITH \"backups=1, affinity_key=city_id\""); + } + + print("Created database objects."); + + // Populate City via COPY command with records from cityBulkLoad.csv + try (Statement stmt = conn.createStatement()) { + stmt.executeUpdate("COPY FROM \"" + + IgniteUtils.resolveIgnitePath("examples/src/main/resources/cityBulkLoad.csv") + "\" " + + "INTO City (id, name) FORMAT CSV"); + } + + // Populate Person via COPY command with records from personBulkLoad.csv + try (Statement stmt = conn.createStatement()) { + stmt.executeUpdate("COPY FROM \"" + + IgniteUtils.resolveIgnitePath("examples/src/main/resources/personBulkLoad.csv") + "\" " + + "INTO Person (id, name, city_id) FORMAT CSV"); + } + + print("Populated data via COPY."); + + // Get data. + try (Statement stmt = conn.createStatement()) { + try (ResultSet rs = + stmt.executeQuery("SELECT p.name, c.name FROM Person p INNER JOIN City c on c.id = p.city_id")) { + print("Query results:"); + + while (rs.next()) + System.out.println(">>> " + rs.getString(1) + ", " + rs.getString(2)); + } + } + + // Drop database objects. + try (Statement stmt = conn.createStatement()) { + stmt.executeUpdate("DROP TABLE Person"); + stmt.executeUpdate("DROP TABLE City"); + } + + print("Dropped database objects."); + } + + print("JDBC COPY example finished."); + } + + /** + * Prints message. + * + * @param msg Message to print before all objects are printed. + */ + private static void print(String msg) { + System.out.println(); + System.out.println(">>> " + msg); + } +} \ No newline at end of file diff --git a/examples/src/main/resources/cityBulkLoad.csv b/examples/src/main/resources/cityBulkLoad.csv new file mode 100644 index 0000000000000..fa3e006d68ed8 --- /dev/null +++ b/examples/src/main/resources/cityBulkLoad.csv @@ -0,0 +1,3 @@ +1,Forest Hill +2,Denver +3,St. Petersburg \ No newline at end of file diff --git a/examples/src/main/resources/personBulkLoad.csv b/examples/src/main/resources/personBulkLoad.csv new file mode 100644 index 0000000000000..c165f974e76d2 --- /dev/null +++ b/examples/src/main/resources/personBulkLoad.csv @@ -0,0 +1,4 @@ +1,John Doe,1 +2,Jane Roe,2 +3,Mary Major,3 +4,Richard Miles,2 \ No newline at end of file From d8203e2d81f8fbf0f7fbe5e710c9908f2fcb8307 Mon Sep 17 00:00:00 2001 From: shq Date: Thu, 15 Feb 2018 13:36:00 +0300 Subject: [PATCH 020/314] IGNITE-7709: SQL COPY command: make sure file name is always quoted. This closes #3526. --- .../sql/command/SqlBulkLoadCommand.java | 8 +++++- .../sql/SqlParserBulkLoadSelfTest.java | 26 ++++++++++++++++--- 2 files changed, 30 insertions(+), 4 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/sql/command/SqlBulkLoadCommand.java b/modules/core/src/main/java/org/apache/ignite/internal/sql/command/SqlBulkLoadCommand.java index e5246d5bd57b5..4f065f20d87a0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/sql/command/SqlBulkLoadCommand.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/sql/command/SqlBulkLoadCommand.java @@ -29,6 +29,7 @@ import java.util.List; import static org.apache.ignite.internal.sql.SqlParserUtils.error; +import static org.apache.ignite.internal.sql.SqlParserUtils.errorUnexpectedToken; import static org.apache.ignite.internal.sql.SqlParserUtils.parseIdentifier; import static org.apache.ignite.internal.sql.SqlParserUtils.parseInt; import static org.apache.ignite.internal.sql.SqlParserUtils.parseQualifiedIdentifier; @@ -83,7 +84,12 @@ public class SqlBulkLoadCommand implements SqlCommand { * @param lex The lexer. */ private void parseFileName(SqlLexer lex) { - locFileName = parseIdentifier(lex); + if (lex.lookAhead().tokenType() != SqlLexerTokenType.QUOTED) + throw errorUnexpectedToken(lex.lookAhead(), "[quoted file name]"); + + lex.shift(); + + locFileName = lex.token(); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/sql/SqlParserBulkLoadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/sql/SqlParserBulkLoadSelfTest.java index b5cd55be2169f..ff09c732fde3b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/sql/SqlParserBulkLoadSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/sql/SqlParserBulkLoadSelfTest.java @@ -33,11 +33,31 @@ public void testCopy() { assertParseError(null, "copy from into Person (_key, age, firstName, lastName) format csv", - "Unexpected token: \"INTO\" (expected: \"[identifier]\""); + "Unexpected token: \"INTO\" (expected: \"[quoted file name]\""); assertParseError(null, - "copy from any.file into Person (_key, age, firstName, lastName) format csv", - "Unexpected token: \".\" (expected: \"INTO\""); + "copy from unquoted into Person (_key, age, firstName, lastName) format csv", + "Unexpected token: \"UNQUOTED\" (expected: \"[quoted file name]\""); + + assertParseError(null, + "copy from unquoted.file into Person (_key, age, firstName, lastName) format csv", + "Unexpected token: \"UNQUOTED\" (expected: \"[quoted file name]\""); + + new SqlParser(null, + "copy from \"\" into Person (_key, age, firstName, lastName) format csv") + .nextCommand(); + + new SqlParser(null, + "copy from \"d:/copy/from/into/format.csv\" into Person (_key, age, firstName, lastName) format csv") + .nextCommand(); + + new SqlParser(null, + "copy from \"/into\" into Person (_key, age, firstName, lastName) format csv") + .nextCommand(); + + new SqlParser(null, + "copy from \"into\" into Person (_key, age, firstName, lastName) format csv") + .nextCommand(); assertParseError(null, "copy from \"any.file\" to Person (_key, age, firstName, lastName) format csv", From 1185993ee7cd83695388f698f18f95b43e15de06 Mon Sep 17 00:00:00 2001 From: devozerov Date: Thu, 15 Feb 2018 14:00:42 +0300 Subject: [PATCH 021/314] IGNITE-7714: SQL COPY command: fixed "Table not found" issue on the client node. --- .../processors/query/h2/DmlStatementsProcessor.java | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java index c46c9062afe19..224bae0115502 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java @@ -1016,9 +1016,16 @@ public FieldsQueryCursor> processBulkLoadCommand(SqlBulkLoadCommand cmd) GridH2Table tbl = idx.dataTable(cmd.schemaName(), cmd.tableName()); - if (tbl == null) + if (tbl == null) { + idx.kernalContext().cache().createMissingQueryCaches(); + + tbl = idx.dataTable(cmd.schemaName(), cmd.tableName()); + } + + if (tbl == null) { throw new IgniteSQLException("Table does not exist: " + cmd.tableName(), IgniteQueryErrorCode.TABLE_NOT_FOUND); + } UpdatePlan plan = UpdatePlanBuilder.planForBulkLoad(cmd, tbl); From 88c8bdcc0dc2fdf2b2b22562a6b30031e053f671 Mon Sep 17 00:00:00 2001 From: devozerov Date: Fri, 16 Feb 2018 17:54:24 +0300 Subject: [PATCH 022/314] IGNITE-7737: SQL COPY: renamed BUFFER_SIZE to PACKET_SIZE. This closes #3533. --- .../JdbcThinBulkLoadAbstractSelfTest.java | 14 +++--- .../internal/jdbc/thin/JdbcThinStatement.java | 2 +- .../bulkload/BulkLoadAckClientParameters.java | 48 +++++++++---------- .../odbc/jdbc/JdbcBulkLoadAckResult.java | 6 +-- .../ignite/internal/sql/SqlKeyword.java | 4 +- .../sql/command/SqlBulkLoadCommand.java | 30 ++++++------ .../query/h2/DmlStatementsProcessor.java | 6 +-- 7 files changed, 55 insertions(+), 55 deletions(-) diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBulkLoadAbstractSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBulkLoadAbstractSelfTest.java index 761f700632880..e37abc4903774 100644 --- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBulkLoadAbstractSelfTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBulkLoadAbstractSelfTest.java @@ -214,17 +214,17 @@ public void testUtf() throws SQLException { } /** - * Imports two-entry CSV file with UTF-8 characters into a table using batch size of one byte + * Imports two-entry CSV file with UTF-8 characters into a table using packet size of one byte * (thus splitting each two-byte UTF-8 character into two batches) * and checks the created entries using SELECT statement. * * @throws SQLException If failed. */ - public void testUtfBatchSize_1() throws SQLException { + public void testUtfPacketSize_1() throws SQLException { int updatesCnt = stmt.executeUpdate( "copy from \"" + BULKLOAD_UTF_CSV_FILE + "\" into " + TBL_NAME + " (_key, age, firstName, lastName)" + - " format csv batch_size 1"); + " format csv packet_size 1"); assertEquals(2, updatesCnt); @@ -388,13 +388,13 @@ public void testConfigureQueryEntityAndBulkLoad() throws SQLException { } /** - * Checks that bulk load works when we use batch size of 1 byte and thus - * create multiple batches per COPY. + * Checks that bulk load works when we use packet size of 1 byte and thus + * create multiple packetes per COPY. * * @throws SQLException If failed. */ - public void testBatchSize_1() throws SQLException { - int updatesCnt = stmt.executeUpdate(BASIC_SQL_COPY_STMT + " batch_size 1"); + public void testPacketSize_1() throws SQLException { + int updatesCnt = stmt.executeUpdate(BASIC_SQL_COPY_STMT + " packet_size 1"); assertEquals(2, updatesCnt); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java index 202001149d932..9c4180439adeb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java @@ -193,7 +193,7 @@ else if (res0 instanceof JdbcQueryExecuteMultipleStatementsResult) { */ private JdbcResult sendFile(JdbcBulkLoadAckResult cmdRes) throws SQLException { String fileName = cmdRes.params().localFileName(); - int batchSize = cmdRes.params().batchSize(); + int batchSize = cmdRes.params().packetSize(); int batchNum = 0; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadAckClientParameters.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadAckClientParameters.java index 119d9f94e9dcf..5a8a859dcc88b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadAckClientParameters.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadAckClientParameters.java @@ -23,33 +23,33 @@ * Bulk load parameters, which are parsed from SQL command and sent from server to client. */ public class BulkLoadAckClientParameters { - /** Minimum batch size. */ - public static final int MIN_BATCH_SIZE = 1; + /** Minimum packet size. */ + public static final int MIN_PACKET_SIZE = 1; /** - * Maximum batch size. Note that the batch is wrapped to transport objects and the overall packet should fit + * Maximum packet size. Note that the packet is wrapped to transport objects and the overall packet should fit * into a Java array. 512 has been chosen arbitrarily. */ - public static final int MAX_BATCH_SIZE = Integer.MAX_VALUE - 512; + public static final int MAX_PACKET_SIZE = Integer.MAX_VALUE - 512; - /** Size of a file batch for COPY command. */ - public static final int DEFAULT_BATCH_SIZE = 4 * 1024 * 1024; + /** Size of a file packet size for COPY command. */ + public static final int DFLT_PACKET_SIZE = 4 * 1024 * 1024; /** Local name of the file to send to server */ @NotNull private final String locFileName; - /** File batch size in bytes. */ - private final int batchSize; + /** File packet size in bytes. */ + private final int packetSize; /** * Creates a bulk load parameters. * * @param locFileName File name to send from client to server. - * @param batchSize Batch size (Number of bytes in a portion of a file to send in one JDBC request/response). + * @param packetSize Packet size (Number of bytes in a portion of a file to send in one JDBC request/response). */ - public BulkLoadAckClientParameters(@NotNull String locFileName, int batchSize) { + public BulkLoadAckClientParameters(@NotNull String locFileName, int packetSize) { this.locFileName = locFileName; - this.batchSize = batchSize; + this.packetSize = packetSize; } /** @@ -62,31 +62,31 @@ public BulkLoadAckClientParameters(@NotNull String locFileName, int batchSize) { } /** - * Returns the batch size. + * Returns the packet size. * - * @return The batch size. + * @return The packet size. */ - public int batchSize() { - return batchSize; + public int packetSize() { + return packetSize; } /** - * Checks if batch size value is valid. + * Checks if packet size value is valid. * - * @param sz The batch size to check. - * @throws IllegalArgumentException if batch size is invalid. + * @param sz The packet size to check. + * @throws IllegalArgumentException if packet size is invalid. */ - public static boolean isValidBatchSize(int sz) { - return sz >= MIN_BATCH_SIZE && sz <= MAX_BATCH_SIZE; + public static boolean isValidPacketSize(int sz) { + return sz >= MIN_PACKET_SIZE && sz <= MAX_PACKET_SIZE; } /** - * Creates proper batch size error message if {@link #isValidBatchSize(int)} check has failed. + * Creates proper packet size error message if {@link #isValidPacketSize(int)} check has failed. * - * @param sz The batch size. + * @param size The packet size. * @return The string with the error message. */ - public static String batchSizeErrorMsg(int sz) { - return "Batch size should be within [" + MIN_BATCH_SIZE + ".." + MAX_BATCH_SIZE + "]: " + sz; + public static String packetSizeErrorMesssage(int size) { + return "Packet size should be within [" + MIN_PACKET_SIZE + ".." + MAX_PACKET_SIZE + "]: " + size; } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBulkLoadAckResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBulkLoadAckResult.java index 8a170ab64b8c7..e670baffcf371 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBulkLoadAckResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBulkLoadAckResult.java @@ -86,7 +86,7 @@ public BulkLoadAckClientParameters params() { writer.writeLong(qryId); writer.writeString(params.localFileName()); - writer.writeInt(params.batchSize()); + writer.writeInt(params.packetSize()); } /** {@inheritDoc} */ @@ -98,8 +98,8 @@ public BulkLoadAckClientParameters params() { String locFileName = reader.readString(); int batchSize = reader.readInt(); - if (!BulkLoadAckClientParameters.isValidBatchSize(batchSize)) - throw new BinaryObjectException(BulkLoadAckClientParameters.batchSizeErrorMsg(batchSize)); + if (!BulkLoadAckClientParameters.isValidPacketSize(batchSize)) + throw new BinaryObjectException(BulkLoadAckClientParameters.packetSizeErrorMesssage(batchSize)); params = new BulkLoadAckClientParameters(locFileName, batchSize); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlKeyword.java b/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlKeyword.java index 0238b0104d625..9a2a865cba464 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlKeyword.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlKeyword.java @@ -42,8 +42,8 @@ public class SqlKeyword { /** Keyword: BOOL. */ public static final String BOOL = "BOOL"; - /** Keyword: BATCH_SIZE. */ - public static final String BATCH_SIZE = "BATCH_SIZE"; + /** Keyword: PACKET_SIZE. */ + public static final String PACKET_SIZE = "PACKET_SIZE"; /** Keyword: BOOLEAN. */ public static final String BOOLEAN = "BOOLEAN"; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/sql/command/SqlBulkLoadCommand.java b/modules/core/src/main/java/org/apache/ignite/internal/sql/command/SqlBulkLoadCommand.java index 4f065f20d87a0..444ebcc6cb35a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/sql/command/SqlBulkLoadCommand.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/sql/command/SqlBulkLoadCommand.java @@ -53,8 +53,8 @@ public class SqlBulkLoadCommand implements SqlCommand { /** File format. */ private BulkLoadFormat inputFormat; - /** Batch size (size of portion of a file sent in each sub-request). */ - private Integer batchSize; + /** Packet size (size of portion of a file sent in each sub-request). */ + private Integer packetSize; /** * Parses the command. @@ -168,15 +168,15 @@ private void parseFormat(SqlLexer lex) { private void parseParameters(SqlLexer lex) { while (lex.lookAhead().tokenType() == SqlLexerTokenType.DEFAULT) { switch (lex.lookAhead().token()) { - case SqlKeyword.BATCH_SIZE: + case SqlKeyword.PACKET_SIZE: lex.shift(); - int sz = parseInt(lex); + int size = parseInt(lex); - if (!BulkLoadAckClientParameters.isValidBatchSize(sz)) - throw error(lex, BulkLoadAckClientParameters.batchSizeErrorMsg(sz)); + if (!BulkLoadAckClientParameters.isValidPacketSize(size)) + throw error(lex, BulkLoadAckClientParameters.packetSizeErrorMesssage(size)); - batchSize = sz; + packetSize = size; break; @@ -255,21 +255,21 @@ public BulkLoadFormat inputFormat() { } /** - * Returns the batch size. + * Returns the packet size. * - * @return The batch size. + * @return The packet size. */ - public Integer batchSize() { - return batchSize; + public Integer packetSize() { + return packetSize; } /** - * Sets the batch size. + * Sets the packet size. * - * @param batchSize The batch size. + * @param packetSize The packet size. */ - public void batchSize(int batchSize) { - this.batchSize = batchSize; + public void packetSize(int packetSize) { + this.packetSize = packetSize; } /** {@inheritDoc} */ diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java index 224bae0115502..ea6c7c1a4a4e6 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java @@ -1011,8 +1011,8 @@ public FieldsQueryCursor> runNativeDmlStatement(String sql, SqlCommand c * @throws IgniteCheckedException If something failed. */ public FieldsQueryCursor> processBulkLoadCommand(SqlBulkLoadCommand cmd) throws IgniteCheckedException { - if (cmd.batchSize() == null) - cmd.batchSize(BulkLoadAckClientParameters.DEFAULT_BATCH_SIZE); + if (cmd.packetSize() == null) + cmd.packetSize(BulkLoadAckClientParameters.DFLT_PACKET_SIZE); GridH2Table tbl = idx.dataTable(cmd.schemaName(), cmd.tableName()); @@ -1041,7 +1041,7 @@ public FieldsQueryCursor> processBulkLoadCommand(SqlBulkLoadCommand cmd) BulkLoadProcessor processor = new BulkLoadProcessor(inputParser, dataConverter, outputWriter); - BulkLoadAckClientParameters params = new BulkLoadAckClientParameters(cmd.localFileName(), cmd.batchSize()); + BulkLoadAckClientParameters params = new BulkLoadAckClientParameters(cmd.localFileName(), cmd.packetSize()); return new BulkLoadContextCursor(processor, params); } From bc331f9de716c30d6f733e28821ab44da7ed0cf7 Mon Sep 17 00:00:00 2001 From: Alexander Paschenko Date: Fri, 16 Feb 2018 23:22:03 +0300 Subject: [PATCH 023/314] IGNITE-7253: Streaming mode for JDBC thin driver. This closes #3499. (cherry picked from commit 692e488) --- .../jdbc2/JdbcNoCacheStreamingSelfTest.java | 182 +++++++++++ .../internal/jdbc2/JdbcStreamingSelfTest.java | 218 +++++++++++--- .../jdbc/suite/IgniteJdbcDriverTestSuite.java | 9 +- .../jdbc/thin/JdbcThinStreamingSelfTest.java | 285 ++++++++++++++++++ .../jdbc/thin/ConnectionProperties.java | 66 ++++ .../jdbc/thin/ConnectionPropertiesImpl.java | 128 +++++++- .../jdbc/thin/JdbcThinConnection.java | 69 +++++ .../jdbc/thin/JdbcThinPreparedStatement.java | 18 +- .../internal/jdbc/thin/JdbcThinStatement.java | 55 +++- .../internal/jdbc/thin/JdbcThinTcpIo.java | 32 +- .../ignite/internal/jdbc2/JdbcConnection.java | 12 +- .../jdbc2/JdbcStreamedPreparedStatement.java | 2 +- .../cache/IgniteCacheProxyImpl.java | 4 +- .../odbc/jdbc/JdbcConnectionContext.java | 29 +- .../odbc/jdbc/JdbcRequestHandler.java | 82 +++-- .../processors/query/GridQueryIndexing.java | 32 +- .../processors/query/GridQueryProcessor.java | 87 +++++- .../processors/query/SqlClientContext.java | 195 ++++++++++++ .../apache/ignite/internal/sql/SqlParser.java | 2 +- ...niteClientCacheInitializationFailTest.java | 15 +- .../ignite/testframework/GridTestUtils.java | 2 +- .../query/h2/DmlStatementsProcessor.java | 104 +++---- .../processors/query/h2/IgniteH2Indexing.java | 121 ++++++-- .../query/h2/ddl/DdlStatementsProcessor.java | 22 +- .../processors/query/h2/dml/UpdatePlan.java | 2 +- .../query/h2/dml/UpdatePlanBuilder.java | 13 +- .../query/h2/sql/GridSqlQueryParser.java | 12 + .../GridCacheCrossCacheQuerySelfTest.java | 18 +- 28 files changed, 1556 insertions(+), 260 deletions(-) create mode 100644 modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcNoCacheStreamingSelfTest.java create mode 100644 modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinStreamingSelfTest.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/query/SqlClientContext.java diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcNoCacheStreamingSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcNoCacheStreamingSelfTest.java new file mode 100644 index 0000000000000..74c2820f06f5a --- /dev/null +++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcNoCacheStreamingSelfTest.java @@ -0,0 +1,182 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.jdbc2; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.PreparedStatement; +import java.util.Collections; +import java.util.Properties; +import org.apache.ignite.IgniteJdbcDriver; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.ConnectorConfiguration; +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.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +import static org.apache.ignite.IgniteJdbcDriver.CFG_URL_PREFIX; +import static org.apache.ignite.cache.CacheMode.PARTITIONED; +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; + +/** + * Data streaming test for thick driver and no explicit caches. + */ +public class JdbcNoCacheStreamingSelfTest extends GridCommonAbstractTest { + /** JDBC URL. */ + private static final String BASE_URL = CFG_URL_PREFIX + + "cache=default@modules/clients/src/test/config/jdbc-config.xml"; + + /** Connection. */ + protected Connection conn; + + /** */ + protected transient IgniteLogger log; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + return getConfiguration0(gridName); + } + + /** + * @param gridName Grid name. + * @return Grid configuration used for starting the grid. + * @throws Exception If failed. + */ + private IgniteConfiguration getConfiguration0(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + CacheConfiguration cache = defaultCacheConfiguration(); + + cache.setCacheMode(PARTITIONED); + cache.setBackups(1); + cache.setWriteSynchronizationMode(FULL_SYNC); + cache.setIndexedTypes( + Integer.class, Integer.class + ); + + cfg.setCacheConfiguration(cache); + cfg.setLocalHost("127.0.0.1"); + + TcpDiscoverySpi disco = new TcpDiscoverySpi(); + + TcpDiscoveryVmIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); + ipFinder.setAddresses(Collections.singleton("127.0.0.1:47500..47501")); + + disco.setIpFinder(ipFinder); + + cfg.setDiscoverySpi(disco); + + cfg.setConnectorConfiguration(new ConnectorConfiguration()); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + startGrids(2); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + } + + /** + * @param allowOverwrite Allow overwriting of existing keys. + * @return Connection to use for the test. + * @throws Exception if failed. + */ + protected Connection createConnection(boolean allowOverwrite) throws Exception { + Properties props = new Properties(); + + props.setProperty(IgniteJdbcDriver.PROP_STREAMING, "true"); + props.setProperty(IgniteJdbcDriver.PROP_STREAMING_FLUSH_FREQ, "500"); + + if (allowOverwrite) + props.setProperty(IgniteJdbcDriver.PROP_STREAMING_ALLOW_OVERWRITE, "true"); + + return DriverManager.getConnection(BASE_URL, props); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + U.closeQuiet(conn); + + ignite(0).cache(DEFAULT_CACHE_NAME).clear(); + + super.afterTest(); + } + + /** + * @throws Exception if failed. + */ + public void testStreamedInsert() throws Exception { + for (int i = 10; i <= 100; i += 10) + ignite(0).cache(DEFAULT_CACHE_NAME).put(i, i * 100); + + try (Connection conn = createConnection(false)) { + try (PreparedStatement stmt = conn.prepareStatement("insert into Integer(_key, _val) values (?, ?)")) { + for (int i = 1; i <= 100; i++) { + stmt.setInt(1, i); + stmt.setInt(2, i); + + stmt.executeUpdate(); + } + } + } + + U.sleep(500); + + // Now let's check it's all there. + for (int i = 1; i <= 100; i++) { + if (i % 10 != 0) + assertEquals(i, grid(0).cache(DEFAULT_CACHE_NAME).get(i)); + else // All that divides by 10 evenly should point to numbers 100 times greater - see above + assertEquals(i * 100, grid(0).cache(DEFAULT_CACHE_NAME).get(i)); + } + } + + /** + * @throws Exception if failed. + */ + public void testStreamedInsertWithOverwritesAllowed() throws Exception { + for (int i = 10; i <= 100; i += 10) + ignite(0).cache(DEFAULT_CACHE_NAME).put(i, i * 100); + + try (Connection conn = createConnection(true)) { + try (PreparedStatement stmt = conn.prepareStatement("insert into Integer(_key, _val) values (?, ?)")) { + for (int i = 1; i <= 100; i++) { + stmt.setInt(1, i); + stmt.setInt(2, i); + + stmt.executeUpdate(); + } + } + } + + U.sleep(500); + + // Now let's check it's all there. + // i should point to i at all times as we've turned overwrites on above. + for (int i = 1; i <= 100; i++) + assertEquals(i, grid(0).cache(DEFAULT_CACHE_NAME).get(i)); + } +} diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcStreamingSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcStreamingSelfTest.java index 5418ca0dd0551..ebb6bc9b340c8 100644 --- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcStreamingSelfTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcStreamingSelfTest.java @@ -20,16 +20,24 @@ import java.sql.Connection; import java.sql.DriverManager; import java.sql.PreparedStatement; +import java.sql.SQLException; +import java.sql.Statement; import java.util.Collections; import java.util.Properties; +import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteJdbcDriver; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.binary.BinaryObject; +import org.apache.ignite.binary.BinaryObjectBuilder; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.ConnectorConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.processors.query.QueryUtils; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteCallable; 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.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import static org.apache.ignite.IgniteJdbcDriver.CFG_URL_PREFIX; @@ -41,10 +49,12 @@ */ public class JdbcStreamingSelfTest extends GridCommonAbstractTest { /** JDBC URL. */ - private static final String BASE_URL = CFG_URL_PREFIX + "cache=default@modules/clients/src/test/config/jdbc-config.xml"; + private static final String BASE_URL = CFG_URL_PREFIX + + "cache=default@modules/clients/src/test/config/jdbc-config.xml"; - /** Connection. */ - protected Connection conn; + /** Streaming URL. */ + private static final String STREAMING_URL = CFG_URL_PREFIX + + "cache=person@modules/clients/src/test/config/jdbc-config.xml"; /** */ protected transient IgniteLogger log; @@ -90,7 +100,18 @@ private IgniteConfiguration getConfiguration0(String gridName) throws Exception /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + startGrids(2); + + try (Connection c = createOrdinaryConnection()) { + try (Statement s = c.createStatement()) { + s.execute("CREATE TABLE PUBLIC.Person(\"id\" int primary key, \"name\" varchar) WITH " + + "\"cache_name=person,value_type=Person\""); + } + } + + U.sleep(1000); } /** {@inheritDoc} */ @@ -98,28 +119,52 @@ private IgniteConfiguration getConfiguration0(String gridName) throws Exception stopAllGrids(); } + /** + * @return Connection without streaming initially turned on. + * @throws SQLException if failed. + */ + protected Connection createOrdinaryConnection() throws SQLException { + Connection res = DriverManager.getConnection(BASE_URL, new Properties()); + + res.setSchema(QueryUtils.DFLT_SCHEMA); + + return res; + } + + /** + * @param allowOverwrite Allow overwriting of existing keys. + * @return Connection to use for the test. + * @throws Exception if failed. + */ + protected Connection createStreamedConnection(boolean allowOverwrite) throws Exception { + return createStreamedConnection(allowOverwrite, 500); + } + /** * @param allowOverwrite Allow overwriting of existing keys. + * @param flushTimeout Stream flush timeout. * @return Connection to use for the test. * @throws Exception if failed. */ - private Connection createConnection(boolean allowOverwrite) throws Exception { + protected Connection createStreamedConnection(boolean allowOverwrite, long flushTimeout) throws Exception { Properties props = new Properties(); props.setProperty(IgniteJdbcDriver.PROP_STREAMING, "true"); - props.setProperty(IgniteJdbcDriver.PROP_STREAMING_FLUSH_FREQ, "500"); + props.setProperty(IgniteJdbcDriver.PROP_STREAMING_FLUSH_FREQ, String.valueOf(flushTimeout)); if (allowOverwrite) props.setProperty(IgniteJdbcDriver.PROP_STREAMING_ALLOW_OVERWRITE, "true"); - return DriverManager.getConnection(BASE_URL, props); + Connection res = DriverManager.getConnection(STREAMING_URL, props); + + res.setSchema(QueryUtils.DFLT_SCHEMA); + + return res; } /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { - U.closeQuiet(conn); - - ignite(0).cache(DEFAULT_CACHE_NAME).clear(); + cache().clear(); super.afterTest(); } @@ -128,30 +173,59 @@ private Connection createConnection(boolean allowOverwrite) throws Exception { * @throws Exception if failed. */ public void testStreamedInsert() throws Exception { - conn = createConnection(false); - for (int i = 10; i <= 100; i += 10) - ignite(0).cache(DEFAULT_CACHE_NAME).put(i, i * 100); + put(i, nameForId(i * 100)); + + try (Connection conn = createStreamedConnection(false)) { + try (PreparedStatement stmt = conn.prepareStatement("insert into PUBLIC.Person(\"id\", \"name\") " + + "values (?, ?)")) { + for (int i = 1; i <= 100; i++) { + stmt.setInt(1, i); + stmt.setString(2, nameForId(i)); + + stmt.executeUpdate(); + } + } + } - PreparedStatement stmt = conn.prepareStatement("insert into Integer(_key, _val) values (?, ?)"); + U.sleep(500); + // Now let's check it's all there. for (int i = 1; i <= 100; i++) { - stmt.setInt(1, i); - stmt.setInt(2, i); + if (i % 10 != 0) + assertEquals(nameForId(i), nameForIdInCache(i)); + else // All that divides by 10 evenly should point to numbers 100 times greater - see above + assertEquals(nameForId(i * 100), nameForIdInCache(i)); + } + } - stmt.executeUpdate(); + /** + * @throws Exception if failed. + */ + public void testStreamedInsertWithoutColumnsList() throws Exception { + for (int i = 10; i <= 100; i += 10) + put(i, nameForId(i * 100)); + + try (Connection conn = createStreamedConnection(false)) { + try (PreparedStatement stmt = conn.prepareStatement("insert into PUBLIC.Person(\"id\", \"name\") " + + "values (?, ?)")) { + for (int i = 1; i <= 100; i++) { + stmt.setInt(1, i); + stmt.setString(2, nameForId(i)); + + stmt.executeUpdate(); + } + } } - // Closing connection makes it wait for streamer close - // and thus for data load completion as well - conn.close(); + U.sleep(500); // Now let's check it's all there. for (int i = 1; i <= 100; i++) { if (i % 10 != 0) - assertEquals(i, grid(0).cache(DEFAULT_CACHE_NAME).get(i)); + assertEquals(nameForId(i), nameForIdInCache(i)); else // All that divides by 10 evenly should point to numbers 100 times greater - see above - assertEquals(i * 100, grid(0).cache(DEFAULT_CACHE_NAME).get(i)); + assertEquals(nameForId(i * 100), nameForIdInCache(i)); } } @@ -159,27 +233,99 @@ public void testStreamedInsert() throws Exception { * @throws Exception if failed. */ public void testStreamedInsertWithOverwritesAllowed() throws Exception { - conn = createConnection(true); - for (int i = 10; i <= 100; i += 10) - ignite(0).cache(DEFAULT_CACHE_NAME).put(i, i * 100); - - PreparedStatement stmt = conn.prepareStatement("insert into Integer(_key, _val) values (?, ?)"); - - for (int i = 1; i <= 100; i++) { - stmt.setInt(1, i); - stmt.setInt(2, i); - - stmt.executeUpdate(); + put(i, nameForId(i * 100)); + + try (Connection conn = createStreamedConnection(true)) { + try (PreparedStatement stmt = conn.prepareStatement("insert into PUBLIC.Person(\"id\", \"name\") " + + "values (?, ?)")) { + for (int i = 1; i <= 100; i++) { + stmt.setInt(1, i); + stmt.setString(2, nameForId(i)); + + stmt.executeUpdate(); + } + } } - // Closing connection makes it wait for streamer close - // and thus for data load completion as well - conn.close(); + U.sleep(500); // Now let's check it's all there. // i should point to i at all times as we've turned overwrites on above. for (int i = 1; i <= 100; i++) - assertEquals(i, grid(0).cache(DEFAULT_CACHE_NAME).get(i)); + assertEquals(nameForId(i), nameForIdInCache(i)); + } + + /** */ + public void testOnlyInsertsAllowed() { + assertStatementForbidden("CREATE TABLE PUBLIC.X (x int primary key, y int)"); + + assertStatementForbidden("SELECT * from Person"); + + assertStatementForbidden("insert into PUBLIC.Person(\"id\", \"name\") " + + "(select \"id\" + 1, CONCAT(\"name\", '1') from Person)"); + + assertStatementForbidden("DELETE from Person"); + + assertStatementForbidden("UPDATE Person SET \"name\" = 'name0'"); + + assertStatementForbidden("alter table Person add column y int"); + } + + /** + * @param sql Statement to check. + */ + @SuppressWarnings("ThrowableNotThrown") + protected void assertStatementForbidden(String sql) { + GridTestUtils.assertThrows(null, new IgniteCallable() { + @Override public Object call() throws Exception { + try (Connection c = createStreamedConnection(false)) { + try (PreparedStatement s = c.prepareStatement(sql)) { + s.execute(); + } + } + + return null; + } + }, SQLException.class,"Only tuple based INSERT statements are supported in streaming mode"); + } + + /** + * @return Person cache. + */ + protected IgniteCache cache() { + return grid(0).cache("person"); + } + + /** + * @param id id of person to put. + * @param name name of person to put. + */ + protected void put(int id, String name) { + BinaryObjectBuilder bldr = grid(0).binary().builder("Person"); + + bldr.setField("name", name); + + cache().put(id, bldr.build()); + } + + /** + * @param id Person id. + * @return Default name for person w/given id. + */ + protected String nameForId(int id) { + return "Person" + id; + } + + /** + * @param id person id. + * @return Name for person with given id currently stored in cache. + */ + protected String nameForIdInCache(int id) { + Object o = cache().withKeepBinary().get(id); + + assertTrue(String.valueOf(o), o instanceof BinaryObject); + + return ((BinaryObject)o).field("name"); } } diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java index 656e218ecee09..c380049db975b 100644 --- a/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java +++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java @@ -42,6 +42,7 @@ import org.apache.ignite.jdbc.thin.JdbcThinBulkLoadTransactionalPartitionedSelfTest; import org.apache.ignite.jdbc.thin.JdbcThinBulkLoadTransactionalReplicatedSelfTest; import org.apache.ignite.jdbc.thin.JdbcThinComplexDmlDdlSelfTest; +import org.apache.ignite.jdbc.thin.JdbcThinComplexDmlDdlSkipReducerOnUpdateSelfTest; import org.apache.ignite.jdbc.thin.JdbcThinComplexQuerySelfTest; import org.apache.ignite.jdbc.thin.JdbcThinConnectionSelfTest; import org.apache.ignite.jdbc.thin.JdbcThinDeleteStatementSelfTest; @@ -54,8 +55,10 @@ import org.apache.ignite.jdbc.thin.JdbcThinEmptyCacheSelfTest; import org.apache.ignite.jdbc.thin.JdbcThinErrorsSelfTest; import org.apache.ignite.jdbc.thin.JdbcThinInsertStatementSelfTest; +import org.apache.ignite.jdbc.thin.JdbcThinInsertStatementSkipReducerOnUpdateSelfTest; import org.apache.ignite.jdbc.thin.JdbcThinLocalQueriesSelfTest; import org.apache.ignite.jdbc.thin.JdbcThinMergeStatementSelfTest; +import org.apache.ignite.jdbc.thin.JdbcThinMergeStatementSkipReducerOnUpdateSelfTest; import org.apache.ignite.jdbc.thin.JdbcThinMetadataSelfTest; import org.apache.ignite.jdbc.thin.JdbcThinMissingLongArrayResultsTest; import org.apache.ignite.jdbc.thin.JdbcThinNoDefaultSchemaTest; @@ -64,10 +67,8 @@ import org.apache.ignite.jdbc.thin.JdbcThinSchemaCaseTest; import org.apache.ignite.jdbc.thin.JdbcThinSelectAfterAlterTable; import org.apache.ignite.jdbc.thin.JdbcThinStatementSelfTest; +import org.apache.ignite.jdbc.thin.JdbcThinStreamingSelfTest; import org.apache.ignite.jdbc.thin.JdbcThinUpdateStatementSelfTest; -import org.apache.ignite.jdbc.thin.JdbcThinComplexDmlDdlSkipReducerOnUpdateSelfTest; -import org.apache.ignite.jdbc.thin.JdbcThinInsertStatementSkipReducerOnUpdateSelfTest; -import org.apache.ignite.jdbc.thin.JdbcThinMergeStatementSkipReducerOnUpdateSelfTest; import org.apache.ignite.jdbc.thin.JdbcThinUpdateStatementSkipReducerOnUpdateSelfTest; import org.apache.ignite.jdbc.thin.JdbcThinWalModeChangeSelfTest; @@ -119,9 +120,11 @@ public static TestSuite suite() throws Exception { suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcStatementBatchingSelfTest.class)); suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcErrorsSelfTest.class)); suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcStreamingToPublicCacheTest.class)); + suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcNoCacheStreamingSelfTest.class)); suite.addTest(new TestSuite(JdbcBlobTest.class)); suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcStreamingSelfTest.class)); + suite.addTest(new TestSuite(JdbcThinStreamingSelfTest.class)); // DDL tests. suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcDynamicIndexAtomicPartitionedNearSelfTest.class)); diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinStreamingSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinStreamingSelfTest.java new file mode 100644 index 0000000000000..9eba4da2a0a8a --- /dev/null +++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinStreamingSelfTest.java @@ -0,0 +1,285 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.jdbc.thin; + +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Set; +import org.apache.ignite.IgniteDataStreamer; +import org.apache.ignite.cache.CachePeekMode; +import org.apache.ignite.internal.jdbc2.JdbcStreamingSelfTest; +import org.apache.ignite.internal.processors.query.SqlClientContext; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.U; + +/** + * Tests for streaming via thin driver. + */ +public class JdbcThinStreamingSelfTest extends JdbcStreamingSelfTest { + /** */ + private int batchSize = 17; + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + batchSize = 17; + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + try (Connection c = createOrdinaryConnection()) { + execute(c, "DROP TABLE PUBLIC.T IF EXISTS"); + } + + super.afterTest(); + } + + /** {@inheritDoc} */ + @Override protected Connection createStreamedConnection(boolean allowOverwrite, long flushFreq) throws Exception { + return JdbcThinAbstractSelfTest.connect(grid(0), "streaming=true&streamingFlushFrequency=" + + flushFreq + "&" + "streamingAllowOverwrite=" + allowOverwrite + "&streamingPerNodeBufferSize=1000&" + + "streamingBatchSize=" + batchSize); + } + + /** {@inheritDoc} */ + @Override protected Connection createOrdinaryConnection() throws SQLException { + return JdbcThinAbstractSelfTest.connect(grid(0), null); + } + + /** + * @throws Exception if failed. + */ + public void testStreamedBatchedInsert() throws Exception { + for (int i = 10; i <= 100; i += 10) + put(i, nameForId(i * 100)); + + try (Connection conn = createStreamedConnection(false)) { + assertStreamingOn(); + + try (PreparedStatement stmt = conn.prepareStatement("insert into Person(\"id\", \"name\") values (?, ?), " + + "(?, ?)")) { + for (int i = 1; i <= 100; i+=2) { + stmt.setInt(1, i); + stmt.setString(2, nameForId(i)); + stmt.setInt(3, i + 1); + stmt.setString(4, nameForId(i + 1)); + + stmt.addBatch(); + } + + stmt.executeBatch(); + } + } + + U.sleep(500); + + // Now let's check it's all there. + for (int i = 1; i <= 100; i++) { + if (i % 10 != 0) + assertEquals(nameForId(i), nameForIdInCache(i)); + else // All that divides by 10 evenly should point to numbers 100 times greater - see above + assertEquals(nameForId(i * 100), nameForIdInCache(i)); + } + } + + /** + * @throws SQLException if failed. + */ + public void testSimultaneousStreaming() throws Exception { + try (Connection anotherConn = createOrdinaryConnection()) { + execute(anotherConn, "CREATE TABLE PUBLIC.T(x int primary key, y int) WITH " + + "\"cache_name=T,wrap_value=false\""); + } + + // Timeout to let connection close be handled on server side. + U.sleep(500); + + try (Connection conn = createStreamedConnection(false, 10000)) { + assertStreamingOn(); + + PreparedStatement firstStmt = conn.prepareStatement("insert into Person(\"id\", \"name\") values (?, ?)"); + + PreparedStatement secondStmt = conn.prepareStatement("insert into PUBLIC.T(x, y) values (?, ?)"); + + try { + for (int i = 1; i <= 10; i++) { + firstStmt.setInt(1, i); + firstStmt.setString(2, nameForId(i)); + + firstStmt.executeUpdate(); + } + + for (int i = 51; i <= 67; i++) { + secondStmt.setInt(1, i); + secondStmt.setInt(2, i); + + secondStmt.executeUpdate(); + } + + for (int i = 11; i <= 50; i++) { + firstStmt.setInt(1, i); + firstStmt.setString(2, nameForId(i)); + + firstStmt.executeUpdate(); + } + + for (int i = 68; i <= 100; i++) { + secondStmt.setInt(1, i); + secondStmt.setInt(2, i); + + secondStmt.executeUpdate(); + } + + assertCacheEmpty(); + + SqlClientContext cliCtx = sqlClientContext(); + + HashMap> streamers = U.field(cliCtx, "streamers"); + + assertEquals(2, streamers.size()); + + assertEqualsCollections(new HashSet<>(Arrays.asList("person", "T")), streamers.keySet()); + } + finally { + U.closeQuiet(firstStmt); + + U.closeQuiet(secondStmt); + } + } + + // Let's wait a little so that all data arrives to destination - we can't intercept streamers' flush + // on connection close in any way. + U.sleep(1000); + + // Now let's check it's all there. + for (int i = 1; i <= 50; i++) + assertEquals(nameForId(i), nameForIdInCache(i)); + + for (int i = 51; i <= 100; i++) + assertEquals(i, grid(0).cache("T").get(i)); + } + + /** + * + */ + public void testStreamingWithMixedStatementTypes() throws Exception { + String prepStmtStr = "insert into Person(\"id\", \"name\") values (?, ?)"; + + String stmtStr = "insert into Person(\"id\", \"name\") values (%d, '%s')"; + + try (Connection conn = createStreamedConnection(false, 10000)) { + assertStreamingOn(); + + PreparedStatement firstStmt = conn.prepareStatement(prepStmtStr); + + Statement secondStmt = conn.createStatement(); + + try { + for (int i = 1; i <= 100; i++) { + boolean usePrep = Math.random() > 0.5; + + boolean useBatch = Math.random() > 0.5; + + if (usePrep) { + firstStmt.setInt(1, i); + firstStmt.setString(2, nameForId(i)); + + if (useBatch) + firstStmt.addBatch(); + else + firstStmt.execute(); + } + else { + String sql = String.format(stmtStr, i, nameForId(i)); + + if (useBatch) + secondStmt.addBatch(sql); + else + secondStmt.execute(sql); + } + } + } + finally { + U.closeQuiet(firstStmt); + + U.closeQuiet(secondStmt); + } + } + + // Let's wait a little so that all data arrives to destination - we can't intercept streamers' flush + // on connection close in any way. + U.sleep(1000); + + // Now let's check it's all there. + for (int i = 1; i <= 100; i++) + assertEquals(nameForId(i), nameForIdInCache(i)); + } + + /** + * Check that there's nothing in cache. + */ + private void assertCacheEmpty() { + assertEquals(0, grid(0).cache(DEFAULT_CACHE_NAME).size(CachePeekMode.ALL)); + } + + /** + * @param conn Connection. + * @param sql Statement. + * @throws SQLException if failed. + */ + private static void execute(Connection conn, String sql) throws SQLException { + try (Statement s = conn.createStatement()) { + s.execute(sql); + } + } + + /** + * @return Active SQL client context. + */ + private SqlClientContext sqlClientContext() { + Set ctxs = U.field(grid(0).context().query(), "cliCtxs"); + + assertFalse(F.isEmpty(ctxs)); + + assertEquals(1, ctxs.size()); + + return ctxs.iterator().next(); + } + + /** + * Check that streaming state on target node is as expected. + */ + private void assertStreamingOn() { + SqlClientContext cliCtx = sqlClientContext(); + + assertTrue(cliCtx.isStream()); + } + + /** {@inheritDoc} */ + @Override protected void assertStatementForbidden(String sql) { + batchSize = 1; + + super.assertStatementForbidden(sql); + } +} \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionProperties.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionProperties.java index d79348415f4cc..458facad23086 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionProperties.java @@ -145,4 +145,70 @@ public interface ConnectionProperties { * @param skipReducerOnUpdate Skip reducer on update flag. */ public void setSkipReducerOnUpdate(boolean skipReducerOnUpdate); + + /** + * @return Streamed connection flag. + */ + public boolean isStream(); + + /** + * @param stream Streamed connection flag. + */ + public void setStream(boolean stream); + + /** + * @return Allow overwrites during streaming connection flag. + */ + public boolean isStreamAllowOverwrite(); + + /** + * @param streamAllowOverwrite Allow overwrites during streaming connection flag. + */ + public void setStreamAllowOverwrite(boolean streamAllowOverwrite); + + /** + * @return Number of parallel operations per node during streaming connection param. + */ + public int getStreamParallelOperations(); + + /** + * @param streamParallelOperations Number of parallel operations per node during streaming connection param. + * @throws SQLException if value check failed. + */ + public void setStreamParallelOperations(int streamParallelOperations) throws SQLException; + + /** + * @return Buffer size during streaming connection param. + */ + public int getStreamBufferSize(); + + /** + * @param streamBufSize Buffer size during streaming connection param. + * @throws SQLException if value check failed. + */ + public void setStreamBufferSize(int streamBufSize) throws SQLException; + + /** + * @return Flush timeout during streaming connection param. + */ + public long getStreamFlushFrequency(); + + /** + * @param streamFlushFreq Flush timeout during streaming connection param. + * @throws SQLException if value check failed. + */ + public void setStreamFlushFrequency(long streamFlushFreq) throws SQLException; + + /** + * @return Batch size for streaming (number of commands to accumulate internally before actually + * sending over the wire). + */ + public int getStreamBatchSize(); + + /** + * @param streamBatchSize Batch size for streaming (number of commands to accumulate internally before actually + * sending over the wire). + * @throws SQLException if value check failed. + */ + public void setStreamBatchSize(int streamBatchSize) throws SQLException; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java index 86ba2fa832b1d..bb0a39d6bf077 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java @@ -96,11 +96,40 @@ public class ConnectionPropertiesImpl implements ConnectionProperties, Serializa private BooleanProperty skipReducerOnUpdate = new BooleanProperty( "skipReducerOnUpdate", "Enable execution update queries on ignite server nodes", false, false); + /** Turn on streaming mode on this connection. */ + private BooleanProperty stream = new BooleanProperty( + "streaming", "Turn on streaming mode on this connection", false, false); + + /** Turn on overwrite during streaming on this connection. */ + private BooleanProperty streamAllowOverwrite = new BooleanProperty( + "streamingAllowOverwrite", "Turn on overwrite during streaming on this connection", false, false); + + /** Number of parallel operations per cluster node during streaming. */ + private IntegerProperty streamParOps = new IntegerProperty( + "streamingPerNodeParallelOperations", "Number of parallel operations per cluster node during streaming", + 0, false, 0, Integer.MAX_VALUE); + + /** Buffer size per cluster node during streaming. */ + private IntegerProperty streamBufSize = new IntegerProperty( + "streamingPerNodeBufferSize", "Buffer size per cluster node during streaming", + 0, false, 0, Integer.MAX_VALUE); + + /** Buffer size per cluster node during streaming. */ + private LongProperty streamFlushFreq = new LongProperty( + "streamingFlushFrequency", "Buffer size per cluster node during streaming", + 0, false, 0, Long.MAX_VALUE); + + /** Buffer size per cluster node during streaming. */ + private IntegerProperty streamBatchSize = new IntegerProperty( + "streamingBatchSize", "Batch size for streaming (number of commands to accumulate internally " + + "before actually sending over the wire)", 10, false, 1, Integer.MAX_VALUE); + /** Properties array. */ private final ConnectionProperty [] propsArray = { host, port, distributedJoins, enforceJoinOrder, collocated, replicatedOnly, autoCloseServerCursor, - tcpNoDelay, lazy, socketSendBuffer, socketReceiveBuffer, skipReducerOnUpdate + tcpNoDelay, lazy, socketSendBuffer, socketReceiveBuffer, skipReducerOnUpdate, + stream, streamAllowOverwrite, streamParOps, streamBufSize, streamFlushFreq, streamBatchSize }; /** {@inheritDoc} */ @@ -223,6 +252,66 @@ public class ConnectionPropertiesImpl implements ConnectionProperties, Serializa skipReducerOnUpdate.setValue(val); } + /** {@inheritDoc} */ + @Override public boolean isStream() { + return stream.value(); + } + + /** {@inheritDoc} */ + @Override public void setStream(boolean val) { + stream.setValue(val); + } + + /** {@inheritDoc} */ + @Override public boolean isStreamAllowOverwrite() { + return streamAllowOverwrite.value(); + } + + /** {@inheritDoc} */ + @Override public void setStreamAllowOverwrite(boolean val) { + streamAllowOverwrite.setValue(val); + } + + /** {@inheritDoc} */ + @Override public int getStreamParallelOperations() { + return streamParOps.value(); + } + + /** {@inheritDoc} */ + @Override public void setStreamParallelOperations(int val) throws SQLException { + streamParOps.setValue(val); + } + + /** {@inheritDoc} */ + @Override public int getStreamBufferSize() { + return streamBufSize.value(); + } + + /** {@inheritDoc} */ + @Override public void setStreamBufferSize(int val) throws SQLException { + streamBufSize.setValue(val); + } + + /** {@inheritDoc} */ + @Override public long getStreamFlushFrequency() { + return streamFlushFreq.value(); + } + + /** {@inheritDoc} */ + @Override public void setStreamFlushFrequency(long val) throws SQLException { + streamFlushFreq.setValue(val); + } + + /** {@inheritDoc} */ + @Override public int getStreamBatchSize() { + return streamBatchSize.value(); + } + + /** {@inheritDoc} */ + @Override public void setStreamBatchSize(int val) throws SQLException { + streamBatchSize.setValue(val); + } + /** * @param props Environment properties. * @throws SQLException On error. @@ -230,7 +319,7 @@ public class ConnectionPropertiesImpl implements ConnectionProperties, Serializa void init(Properties props) throws SQLException { Properties props0 = (Properties)props.clone(); - for (ConnectionProperty aPropsArray : propsArray) + for (ConnectionProperty aPropsArray : this.propsArray) aPropsArray.init(props0); } @@ -513,7 +602,8 @@ private abstract static class NumberProperty extends ConnectionProperty { else { try { setValue(parse(str)); - } catch (NumberFormatException e) { + } + catch (NumberFormatException e) { throw new SQLException("Failed to parse int property [name=" + name + ", value=" + str + ']', SqlStateCode.CLIENT_CONNECTION_FAILED); } @@ -585,6 +675,38 @@ int value() { } } + /** + * + */ + private static class LongProperty extends NumberProperty { + /** */ + private static final long serialVersionUID = 0L; + + /** + * @param name Name. + * @param desc Description. + * @param dfltVal Default value. + * @param required {@code true} if the property is required. + * @param min Lower bound of allowed range. + * @param max Upper bound of allowed range. + */ + LongProperty(String name, String desc, Number dfltVal, boolean required, long min, long max) { + super(name, desc, dfltVal, required, min, max); + } + + /** {@inheritDoc} */ + @Override protected Number parse(String str) throws NumberFormatException { + return Long.parseLong(str); + } + + /** + * @return Property value. + */ + long value() { + return val.longValue(); + } + } + /** * */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java index 999c793d55ccf..d267070718f28 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java @@ -18,6 +18,7 @@ package org.apache.ignite.internal.jdbc.thin; import java.sql.Array; +import java.sql.BatchUpdateException; import java.sql.Blob; import java.sql.CallableStatement; import java.sql.Clob; @@ -33,13 +34,19 @@ import java.sql.Savepoint; import java.sql.Statement; import java.sql.Struct; +import java.util.ArrayList; +import java.util.List; import java.util.Map; import java.util.Properties; import java.util.concurrent.Executor; +import java.util.logging.Level; import java.util.logging.Logger; import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode; import org.apache.ignite.internal.processors.odbc.ClientListenerResponse; import org.apache.ignite.internal.processors.odbc.SqlStateCode; +import org.apache.ignite.internal.processors.odbc.jdbc.JdbcBatchExecuteRequest; +import org.apache.ignite.internal.processors.odbc.jdbc.JdbcBatchExecuteResult; +import org.apache.ignite.internal.processors.odbc.jdbc.JdbcQuery; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcResponse; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcResult; @@ -94,6 +101,12 @@ public class JdbcThinConnection implements Connection { /** Connection properties. */ private ConnectionProperties connProps; + /** Batch for streaming. */ + private List streamBatch; + + /** Last added query to recognize batches. */ + private String lastStreamQry; + /** * Creates new connection. * @@ -130,6 +143,53 @@ public JdbcThinConnection(String url, String schema, Properties props) throws SQ } } + /** + * @return Whether this connection is streamed or not. + */ + public boolean isStream() { + return connProps.isStream(); + } + + /** + * Add another query for batched execution. + * @param sql Query. + * @param args Arguments. + */ + synchronized void addBatch(String sql, List args) throws SQLException { + boolean newQry = (args == null || !F.eq(lastStreamQry, sql)); + + // Providing null as SQL here allows for recognizing subbatches on server and handling them more efficiently. + JdbcQuery q = new JdbcQuery(newQry ? sql : null, args != null ? args.toArray() : null); + + if (streamBatch == null) + streamBatch = new ArrayList<>(connProps.getStreamBatchSize()); + + streamBatch.add(q); + + // Null args means "addBatch(String)" was called on non-prepared Statement, + // we don't want to remember its query string. + lastStreamQry = (args != null ? sql : null); + + if (streamBatch.size() == connProps.getStreamBatchSize()) + executeBatch(); + } + + /** + * @throws SQLException if failed. + */ + private void executeBatch() throws SQLException { + JdbcBatchExecuteResult res = sendRequest(new JdbcBatchExecuteRequest(schema, streamBatch)); + + streamBatch = null; + + lastStreamQry = null; + + if (res.errorCode() != ClientListenerResponse.STATUS_SUCCESS) { + throw new BatchUpdateException(res.errorMessage(), IgniteQueryErrorCode.codeToSqlState(res.errorCode()), + res.errorCode(), res.updateCounts()); + } + } + /** {@inheritDoc} */ @Override public Statement createStatement() throws SQLException { return createStatement(TYPE_FORWARD_ONLY, CONCUR_READ_ONLY, HOLD_CURSORS_OVER_COMMIT); @@ -272,6 +332,15 @@ private void checkCursorOptions(int resSetType, int resSetConcurrency, if (isClosed()) return; + if (!F.isEmpty(streamBatch)) { + try { + executeBatch(); + } + catch (SQLException e) { + LOG.log(Level.WARNING, "Exception during batch send on streamed connection close", e); + } + } + closed = true; cliIo.close(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinPreparedStatement.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinPreparedStatement.java index 23d3bbe37102b..b575167ea8965 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinPreparedStatement.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinPreparedStatement.java @@ -39,8 +39,8 @@ import java.sql.Timestamp; import java.util.ArrayList; import java.util.Calendar; -import org.apache.ignite.internal.processors.odbc.SqlStateCode; import org.apache.ignite.internal.processors.odbc.SqlListenerUtils; +import org.apache.ignite.internal.processors.odbc.SqlStateCode; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMetaParamsRequest; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMetaParamsResult; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcQuery; @@ -262,13 +262,19 @@ private void executeWithArguments(JdbcStatementType stmtType) throws SQLExceptio @Override public void addBatch() throws SQLException { ensureNotClosed(); - if (batch == null) { - batch = new ArrayList<>(); + batchSize++; + + if (conn.isStream()) + conn.addBatch(sql, args); + else { + if (batch == null) { + batch = new ArrayList<>(); - batch.add(new JdbcQuery(sql, args.toArray(new Object[args.size()]))); + batch.add(new JdbcQuery(sql, args.toArray(new Object[args.size()]))); + } + else + batch.add(new JdbcQuery(null, args.toArray(new Object[args.size()]))); } - else - batch.add(new JdbcQuery(null, args.toArray(new Object[args.size()]))); args = null; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java index 9c4180439adeb..61d837c1d2206 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java @@ -38,13 +38,13 @@ import org.apache.ignite.internal.processors.odbc.jdbc.JdbcBatchExecuteRequest; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcBatchExecuteResult; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcBulkLoadAckResult; +import org.apache.ignite.internal.processors.odbc.jdbc.JdbcBulkLoadBatchRequest; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcQuery; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcQueryExecuteMultipleStatementsResult; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcQueryExecuteRequest; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcQueryExecuteResult; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcResult; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcResultInfo; -import org.apache.ignite.internal.processors.odbc.jdbc.JdbcBulkLoadBatchRequest; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcStatementType; import static java.sql.ResultSet.CONCUR_READ_ONLY; @@ -79,6 +79,9 @@ public class JdbcThinStatement implements Statement { /** Result set holdability*/ private final int resHoldability; + /** Batch size to keep track of number of items to return as fake update counters for executeBatch. */ + protected int batchSize; + /** Batch. */ protected List batch; @@ -133,6 +136,19 @@ protected void execute0(JdbcStatementType stmtType, String sql, List arg if (sql == null || sql.isEmpty()) throw new SQLException("SQL query is empty."); + if (conn.isStream()) { + if (stmtType == JdbcStatementType.SELECT_STATEMENT_TYPE) + throw new SQLException("Only tuple based INSERT statements are supported in streaming mode.", + SqlStateCode.INTERNAL_ERROR, + IgniteQueryErrorCode.UNSUPPORTED_OPERATION); + + conn.addBatch(sql, args); + + resultSets = Collections.singletonList(resultSetForUpdate(0)); + + return; + } + JdbcResult res0 = conn.sendRequest(new JdbcQueryExecuteRequest(stmtType, schema, pageSize, maxRows, sql, args == null ? null : args.toArray(new Object[args.size()]))); @@ -158,11 +174,8 @@ else if (res0 instanceof JdbcQueryExecuteMultipleStatementsResult) { boolean firstRes = true; for(JdbcResultInfo rsInfo : resInfos) { - if (!rsInfo.isQuery()) { - resultSets.add(new JdbcThinResultSet(this, -1, pageSize, - true, Collections.>emptyList(), false, - conn.autoCloseServerCursor(), rsInfo.updateCount(), closeOnCompletion)); - } + if (!rsInfo.isQuery()) + resultSets.add(resultSetForUpdate(rsInfo.updateCount())); else { if (firstRes) { firstRes = false; @@ -185,6 +198,16 @@ else if (res0 instanceof JdbcQueryExecuteMultipleStatementsResult) { assert resultSets.size() > 0 : "At least one results set is expected"; } + /** + * @param cnt Update counter. + * @return Result set for given update counter. + */ + private JdbcThinResultSet resultSetForUpdate(long cnt) { + return new JdbcThinResultSet(this, -1, pageSize, + true, Collections.>emptyList(), false, + conn.autoCloseServerCursor(), cnt, closeOnCompletion); + } + /** * Sends a file to server in batches via multiple {@link JdbcBulkLoadBatchRequest}s. * @@ -469,6 +492,14 @@ private JdbcThinResultSet nextResultSet() throws SQLException { @Override public void addBatch(String sql) throws SQLException { ensureNotClosed(); + batchSize++; + + if (conn.isStream()) { + conn.addBatch(sql, null); + + return; + } + if (batch == null) batch = new ArrayList<>(); @@ -479,6 +510,8 @@ private JdbcThinResultSet nextResultSet() throws SQLException { @Override public void clearBatch() throws SQLException { ensureNotClosed(); + batchSize = 0; + batch = null; } @@ -488,6 +521,14 @@ private JdbcThinResultSet nextResultSet() throws SQLException { closeResults(); + if (conn.isStream()) { + int[] res = new int[batchSize]; + + batchSize = 0; + + return res; + } + if (batch == null || batch.isEmpty()) throw new SQLException("Batch is empty."); @@ -502,6 +543,8 @@ private JdbcThinResultSet nextResultSet() throws SQLException { return res.updateCounts(); } finally { + batchSize = 0; + batch = null; } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java index fec218e368494..79bdc5e9189c7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java @@ -20,27 +20,9 @@ import java.io.BufferedInputStream; import java.io.BufferedOutputStream; import java.io.IOException; -import java.io.InputStream; import java.net.InetSocketAddress; -import java.net.MalformedURLException; import java.net.Socket; -import java.net.URL; -import java.security.KeyManagementException; -import java.security.KeyStore; -import java.security.KeyStoreException; -import java.security.NoSuchAlgorithmException; -import java.security.UnrecoverableKeyException; -import java.security.cert.CertificateException; import java.sql.SQLException; -import java.util.ArrayList; -import java.util.List; -import javax.net.ssl.KeyManager; -import javax.net.ssl.KeyManagerFactory; -import javax.net.ssl.SSLContext; -import javax.net.ssl.SSLSocketFactory; -import javax.net.ssl.TrustManager; -import javax.net.ssl.TrustManagerFactory; -import javax.net.ssl.X509TrustManager; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; @@ -57,7 +39,6 @@ import org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcResponse; import org.apache.ignite.internal.util.ipc.loopback.IpcClientTcpEndpoint; -import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteProductVersion; @@ -77,8 +58,11 @@ public class JdbcThinTcpIo { /** Version 2.4.0. */ private static final ClientListenerProtocolVersion VER_2_4_0 = ClientListenerProtocolVersion.create(2, 4, 0); + /** Version 2.5.0. */ + private static final ClientListenerProtocolVersion VER_2_5_0 = ClientListenerProtocolVersion.create(2, 5, 0); + /** Current version. */ - private static final ClientListenerProtocolVersion CURRENT_VER = VER_2_4_0; + private static final ClientListenerProtocolVersion CURRENT_VER = VER_2_5_0; /** Initial output stream capacity for handshake. */ private static final int HANDSHAKE_MSG_SIZE = 13; @@ -182,6 +166,11 @@ public void handshake(ClientListenerProtocolVersion ver) throws IOException, SQL writer.writeBoolean(connProps.isAutoCloseServerCursor()); writer.writeBoolean(connProps.isLazy()); writer.writeBoolean(connProps.isSkipReducerOnUpdate()); + writer.writeBoolean(connProps.isStream()); + writer.writeBoolean(connProps.isStreamAllowOverwrite()); + writer.writeInt(connProps.getStreamParallelOperations()); + writer.writeInt(connProps.getStreamBufferSize()); + writer.writeLong(connProps.getStreamFlushFrequency()); send(writer.array()); @@ -215,7 +204,8 @@ public void handshake(ClientListenerProtocolVersion ver) throws IOException, SQL ClientListenerProtocolVersion srvProtocolVer = ClientListenerProtocolVersion.create(maj, min, maintenance); - if (VER_2_3_0.equals(srvProtocolVer) || VER_2_1_5.equals(srvProtocolVer)) + if (VER_2_4_0.equals(srvProtocolVer) || VER_2_3_0.equals(srvProtocolVer) || + VER_2_1_5.equals(srvProtocolVer)) handshake(srvProtocolVer); else if (VER_2_1_0.equals(srvProtocolVer)) handshake_2_1_0(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java index b51e0b95084ef..aaa51a0db6ca7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java @@ -60,6 +60,7 @@ import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode; import org.apache.ignite.internal.processors.odbc.SqlStateCode; import org.apache.ignite.internal.processors.query.GridQueryIndexing; +import org.apache.ignite.internal.processors.query.IgniteSQLException; import org.apache.ignite.internal.processors.query.QueryUtils; import org.apache.ignite.internal.processors.resource.GridSpringResourceContext; import org.apache.ignite.internal.util.future.GridFutureAdapter; @@ -82,13 +83,13 @@ import static org.apache.ignite.IgniteJdbcDriver.PROP_LOCAL; import static org.apache.ignite.IgniteJdbcDriver.PROP_MULTIPLE_STMTS; import static org.apache.ignite.IgniteJdbcDriver.PROP_NODE_ID; +import static org.apache.ignite.IgniteJdbcDriver.PROP_SKIP_REDUCER_ON_UPDATE; import static org.apache.ignite.IgniteJdbcDriver.PROP_STREAMING; import static org.apache.ignite.IgniteJdbcDriver.PROP_STREAMING_ALLOW_OVERWRITE; import static org.apache.ignite.IgniteJdbcDriver.PROP_STREAMING_FLUSH_FREQ; import static org.apache.ignite.IgniteJdbcDriver.PROP_STREAMING_PER_NODE_BUF_SIZE; import static org.apache.ignite.IgniteJdbcDriver.PROP_STREAMING_PER_NODE_PAR_OPS; import static org.apache.ignite.IgniteJdbcDriver.PROP_TX_ALLOWED; -import static org.apache.ignite.IgniteJdbcDriver.PROP_SKIP_REDUCER_ON_UPDATE; import static org.apache.ignite.internal.jdbc2.JdbcUtils.convertToSqlException; import static org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode.createJdbcSqlException; @@ -612,10 +613,11 @@ private Ignite getIgnite(String cfgUrl) throws IgniteCheckedException { PreparedStatement nativeStmt = prepareNativeStatement(sql); - if (!idx.isInsertStatement(nativeStmt)) { - throw new SQLException("Only INSERT operations are supported in streaming mode", - SqlStateCode.INTERNAL_ERROR, - IgniteQueryErrorCode.UNSUPPORTED_OPERATION); + try { + idx.checkStatementStreamable(nativeStmt); + } + catch (IgniteSQLException e) { + throw e.toJdbcException(); } IgniteDataStreamer streamer = ignite().dataStreamer(cacheName); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStreamedPreparedStatement.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStreamedPreparedStatement.java index 408f0897cad6f..25f55f2f6d2cd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStreamedPreparedStatement.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStreamedPreparedStatement.java @@ -55,7 +55,7 @@ class JdbcStreamedPreparedStatement extends JdbcPreparedStatement { /** {@inheritDoc} */ @Override protected void execute0(String sql, Boolean isQuery) throws SQLException { - assert isQuery != null && !isQuery; + assert isQuery == null || !isQuery; long updCnt = conn.ignite().context().query().streamUpdateQuery(conn.cacheName(), conn.schemaName(), streamer, sql, getArgs()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java index 7f71c743769bf..735a4decf62f2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java @@ -571,7 +571,7 @@ private QueryCursor> queryContinuous(ContinuousQuery qry, bool boolean keepBinary = opCtxCall != null && opCtxCall.isKeepBinary(); - return ctx.kernalContext().query().querySqlFields(ctx, qry, keepBinary, false); + return ctx.kernalContext().query().querySqlFields(ctx, qry, null, keepBinary, false); } catch (Exception e) { if (e instanceof CacheException) @@ -604,7 +604,7 @@ private QueryCursor> queryContinuous(ContinuousQuery qry, bool if (qry instanceof SqlFieldsQuery) return (FieldsQueryCursor)ctx.kernalContext().query().querySqlFields(ctx, (SqlFieldsQuery)qry, - keepBinary, true).get(0); + null, keepBinary, true).get(0); if (qry instanceof ScanQuery) return query((ScanQuery)qry, null, projection(qry.isLocal())); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java index 5841a4d450e9f..214d006da9956 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java @@ -28,7 +28,7 @@ import org.apache.ignite.internal.util.GridSpinBusyLock; /** - * ODBC Connection Context. + * JDBC Connection Context. */ public class JdbcConnectionContext implements ClientListenerConnectionContext { /** Version 2.1.0. */ @@ -38,13 +38,16 @@ public class JdbcConnectionContext implements ClientListenerConnectionContext { private static final ClientListenerProtocolVersion VER_2_1_5 = ClientListenerProtocolVersion.create(2, 1, 5); /** Version 2.3.1: added "multiple statements query" feature. */ - public static final ClientListenerProtocolVersion VER_2_3_0 = ClientListenerProtocolVersion.create(2, 3, 0); + static final ClientListenerProtocolVersion VER_2_3_0 = ClientListenerProtocolVersion.create(2, 3, 0); /** Version 2.4.0: adds default values for columns feature. */ - public static final ClientListenerProtocolVersion VER_2_4_0 = ClientListenerProtocolVersion.create(2, 4, 0); + static final ClientListenerProtocolVersion VER_2_4_0 = ClientListenerProtocolVersion.create(2, 4, 0); + + /** Version 2.5.0: adds streaming via thin connection. */ + static final ClientListenerProtocolVersion VER_2_5_0 = ClientListenerProtocolVersion.create(2, 5, 0); /** Current version. */ - private static final ClientListenerProtocolVersion CURRENT_VER = VER_2_4_0; + private static final ClientListenerProtocolVersion CURRENT_VER = VER_2_5_0; /** Supported versions. */ private static final Set SUPPORTED_VERS = new HashSet<>(); @@ -66,6 +69,7 @@ public class JdbcConnectionContext implements ClientListenerConnectionContext { static { SUPPORTED_VERS.add(CURRENT_VER); + SUPPORTED_VERS.add(VER_2_4_0); SUPPORTED_VERS.add(VER_2_3_0); SUPPORTED_VERS.add(VER_2_1_5); SUPPORTED_VERS.add(VER_2_1_0); @@ -113,8 +117,23 @@ public JdbcConnectionContext(GridKernalContext ctx, GridSpinBusyLock busyLock, i if (ver.compareTo(VER_2_3_0) >= 0) skipReducerOnUpdate = reader.readBoolean(); + boolean stream = false; + boolean streamAllowOverwrites = false; + int streamParOps = 0; + int streamBufSize = 0; + long streamFlushFreq = 0; + + if (ver.compareTo(VER_2_5_0) >= 0) { + stream = reader.readBoolean(); + streamAllowOverwrites = reader.readBoolean(); + streamParOps = reader.readInt(); + streamBufSize = reader.readInt(); + streamFlushFreq = reader.readLong(); + } + handler = new JdbcRequestHandler(ctx, busyLock, maxCursors, distributedJoins, enforceJoinOrder, - collocated, replicatedOnly, autoCloseCursors, lazyExec, skipReducerOnUpdate, ver); + collocated, replicatedOnly, autoCloseCursors, lazyExec, skipReducerOnUpdate, stream, streamAllowOverwrites, + streamParOps, streamBufSize, streamFlushFreq, ver); parser = new JdbcMessageParser(ctx); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java index 59fc06bcd02c7..8786f26bf21b1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java @@ -53,6 +53,7 @@ import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor; import org.apache.ignite.internal.processors.query.IgniteSQLException; import org.apache.ignite.internal.processors.query.QueryUtils; +import org.apache.ignite.internal.processors.query.SqlClientContext; import org.apache.ignite.internal.util.GridSpinBusyLock; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.X; @@ -88,6 +89,9 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler { /** Kernel context. */ private final GridKernalContext ctx; + /** Client context. */ + private final SqlClientContext cliCtx; + /** Logger. */ private final IgniteLogger log; @@ -103,24 +107,9 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler { /** Current bulk load processors. */ private final ConcurrentHashMap bulkLoadRequests = new ConcurrentHashMap<>(); - /** Distributed joins flag. */ - private final boolean distributedJoins; - - /** Enforce join order flag. */ - private final boolean enforceJoinOrder; - - /** Collocated flag. */ - private final boolean collocated; - /** Replicated only flag. */ private final boolean replicatedOnly; - /** Lazy query execution flag. */ - private final boolean lazy; - - /** Skip reducer on update flag. */ - private final boolean skipReducerOnUpdate; - /** Automatic close of cursors. */ private final boolean autoCloseCursors; @@ -140,22 +129,38 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler { * @param autoCloseCursors Flag to automatically close server cursors. * @param lazy Lazy query execution flag. * @param skipReducerOnUpdate Skip reducer on update flag. + * @param stream Streaming flag. + * @param streamAllowOverwrites Streaming overwrites flag. + * @param streamParOps Number of parallel ops per cluster node during streaming. + * @param streamBufSize Buffer size per cluster node during streaming. + * @param streamFlushFreq Data streamers' flush timeout. * @param protocolVer Protocol version. */ public JdbcRequestHandler(GridKernalContext ctx, GridSpinBusyLock busyLock, int maxCursors, boolean distributedJoins, boolean enforceJoinOrder, boolean collocated, boolean replicatedOnly, boolean autoCloseCursors, boolean lazy, boolean skipReducerOnUpdate, + boolean stream, boolean streamAllowOverwrites, int streamParOps, int streamBufSize, long streamFlushFreq, ClientListenerProtocolVersion protocolVer) { this.ctx = ctx; + + this.cliCtx = new SqlClientContext( + ctx, + distributedJoins, + enforceJoinOrder, + collocated, + lazy, + skipReducerOnUpdate, + stream, + streamAllowOverwrites, + streamParOps, + streamBufSize, + streamFlushFreq + ); + this.busyLock = busyLock; this.maxCursors = maxCursors; - this.distributedJoins = distributedJoins; - this.enforceJoinOrder = enforceJoinOrder; - this.collocated = collocated; this.replicatedOnly = replicatedOnly; this.autoCloseCursors = autoCloseCursors; - this.lazy = lazy; - this.skipReducerOnUpdate = skipReducerOnUpdate; this.protocolVer = protocolVer; log = ctx.log(getClass()); @@ -301,6 +306,8 @@ public void onDisconnect() { } bulkLoadRequests.clear(); + + U.close(cliCtx, log); } finally { busyLock.leaveBusy(); @@ -326,6 +333,8 @@ private JdbcResponse executeQuery(JdbcQueryExecuteRequest req) { long qryId = QRY_ID_GEN.getAndIncrement(); + assert !cliCtx.isStream(); + try { String sql = req.sqlQuery(); @@ -347,17 +356,17 @@ private JdbcResponse executeQuery(JdbcQueryExecuteRequest req) { qry = new SqlFieldsQueryEx(sql, false); - if (skipReducerOnUpdate) + if (cliCtx.isSkipReducerOnUpdate()) ((SqlFieldsQueryEx)qry).setSkipReducerOnUpdate(true); } qry.setArgs(req.arguments()); - qry.setDistributedJoins(distributedJoins); - qry.setEnforceJoinOrder(enforceJoinOrder); - qry.setCollocated(collocated); + qry.setDistributedJoins(cliCtx.isDistributedJoins()); + qry.setEnforceJoinOrder(cliCtx.isEnforceJoinOrder()); + qry.setCollocated(cliCtx.isCollocated()); qry.setReplicatedOnly(replicatedOnly); - qry.setLazy(lazy); + qry.setLazy(cliCtx.isLazy()); if (req.pageSize() <= 0) return new JdbcResponse(IgniteQueryErrorCode.UNKNOWN, "Invalid fetch size: " + req.pageSize()); @@ -371,7 +380,7 @@ private JdbcResponse executeQuery(JdbcQueryExecuteRequest req) { qry.setSchema(schemaName); - List>> results = ctx.query().querySqlFields(qry, true, + List>> results = ctx.query().querySqlFields(null, qry, cliCtx, true, protocolVer.compareTo(VER_2_3_0) < 0); FieldsQueryCursor> fieldsCur = results.get(0); @@ -569,11 +578,11 @@ private ClientListenerResponse executeBatch(JdbcBatchExecuteRequest req) { qry = new SqlFieldsQueryEx(q.sql(), false); - qry.setDistributedJoins(distributedJoins); - qry.setEnforceJoinOrder(enforceJoinOrder); - qry.setCollocated(collocated); + qry.setDistributedJoins(cliCtx.isDistributedJoins()); + qry.setEnforceJoinOrder(cliCtx.isEnforceJoinOrder()); + qry.setCollocated(cliCtx.isCollocated()); qry.setReplicatedOnly(replicatedOnly); - qry.setLazy(lazy); + qry.setLazy(cliCtx.isLazy()); qry.setSchema(schemaName); } @@ -601,10 +610,21 @@ private ClientListenerResponse executeBatch(JdbcBatchExecuteRequest req) { * @param updCntsAcc Per query rows updates counter. * @param firstErr First error data - code and message. */ + @SuppressWarnings("ForLoopReplaceableByForEach") private void executeBatchedQuery(SqlFieldsQueryEx qry, List updCntsAcc, IgniteBiTuple firstErr) { try { - List>> qryRes = ctx.query().querySqlFields(qry, true, true); + if (cliCtx.isStream()) { + List cnt = ctx.query().streamBatchedUpdateQuery(qry.getSchema(), cliCtx, qry.getSql(), + qry.batchedArguments()); + + for (int i = 0; i < cnt.size(); i++) + updCntsAcc.add(cnt.get(i).intValue()); + + return; + } + + List>> qryRes = ctx.query().querySqlFields(null, qry, cliCtx, true, true); for (FieldsQueryCursor> cur : qryRes) { if (cur instanceof BulkLoadContextCursor) 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 6b425a1ce4a58..dedd0752f8684 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 @@ -76,33 +76,46 @@ public QueryCursor> queryDistributedSql(String schemaNa * Detect whether SQL query should be executed in distributed or local manner and execute it. * @param schemaName Schema name. * @param qry Query. + * @param cliCtx Client context. * @param keepBinary Keep binary flag. * @param failOnMultipleStmts Whether an exception should be thrown for multiple statements query. - * @param cancel Query cancel state handler. - * @return Cursor. + * @param cancel Query cancel state handler. @return Cursor. */ - public List>> querySqlFields(String schemaName, SqlFieldsQuery qry, boolean keepBinary, - boolean failOnMultipleStmts, GridQueryCancel cancel); + public List>> querySqlFields(String schemaName, SqlFieldsQuery qry, + SqlClientContext cliCtx, boolean keepBinary, boolean failOnMultipleStmts, GridQueryCancel cancel); /** - * Perform a MERGE statement using data streamer as receiver. + * Execute an INSERT statement using data streamer as receiver. * * @param schemaName Schema name. * @param qry Query. * @param params Query parameters. * @param streamer Data streamer to feed data to. - * @return Query result. + * @return Update counter. * @throws IgniteCheckedException If failed. */ public long streamUpdateQuery(String schemaName, String qry, @Nullable Object[] params, IgniteDataStreamer streamer) throws IgniteCheckedException; + /** + * Execute a batched INSERT statement using data streamer as receiver. + * + * @param schemaName Schema name. + * @param qry Query. + * @param params Query parameters. + * @param cliCtx Client connection context. + * @return Update counters. + * @throws IgniteCheckedException If failed. + */ + public List streamBatchedUpdateQuery(String schemaName, String qry, List params, + SqlClientContext cliCtx) throws IgniteCheckedException; + /** * Executes regular query. * * @param schemaName Schema name. * @param cacheName Cache name. - *@param qry Query. + * @param qry Query. * @param filter Cache name and key filter. * @param keepBinary Keep binary flag. @return Cursor. */ @@ -313,12 +326,11 @@ public void remove(GridCacheContext cctx, GridQueryTypeDescriptor type, CacheDat public String schema(String cacheName); /** - * Check if passed statement is insert statemtn. + * Check if passed statement is insert statement eligible for streaming, throw an {@link IgniteSQLException} if not. * * @param nativeStmt Native statement. - * @return {@code True} if insert. */ - public boolean isInsertStatement(PreparedStatement nativeStmt); + public void checkStatementStreamable(PreparedStatement nativeStmt); /** * Return row cache cleaner. 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 43bed4026ed2f..0c73b05edd8db 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 @@ -64,9 +64,7 @@ import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor; import org.apache.ignite.internal.processors.cache.GridCacheAdapter; 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.StoredCacheData; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.processors.cache.query.CacheQueryFuture; import org.apache.ignite.internal.processors.cache.query.CacheQueryType; @@ -102,6 +100,7 @@ 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.A; 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; @@ -191,6 +190,9 @@ public class GridQueryProcessor extends GridProcessorAdapter { /** Pending status messages. */ private final LinkedList pendingMsgs = new LinkedList<>(); + /** All currently open client contexts. */ + private final Set cliCtxs = Collections.newSetFromMap(new ConcurrentHashMap<>()); + /** Current cache that has a query running on it. */ private final ThreadLocal curCache = new ThreadLocal<>(); @@ -261,11 +263,15 @@ public GridQueryProcessor(GridKernalContext ctx) throws IgniteCheckedException { if (cancel && idx != null) { try { - while (!busyLock.tryBlock(500)) + while (!busyLock.tryBlock(500)) { idx.cancelAllQueries(); + closeAllSqlStreams(); + } + return; - } catch (InterruptedException ignored) { + } + catch (InterruptedException ignored) { U.warn(log, "Interrupted while waiting for active queries cancellation."); Thread.currentThread().interrupt(); @@ -348,6 +354,32 @@ public void onCacheReconnect() throws IgniteCheckedException { } } + /** + * @param cliCtx Client context to register. + */ + void registerClientContext(SqlClientContext cliCtx) { + A.notNull(cliCtx, "cliCtx"); + + cliCtxs.add(cliCtx); + } + + /** + * @param cliCtx Client context to register. + */ + void unregisterClientContext(SqlClientContext cliCtx) { + A.notNull(cliCtx, "cliCtx"); + + cliCtxs.remove(cliCtx); + } + + /** + * Flush streamers on all currently open client contexts. + */ + private void closeAllSqlStreams() { + for (SqlClientContext cliCtx : cliCtxs) + U.close(cliCtx, log); + } + /** * Process schema propose message from discovery thread. * @@ -1976,13 +2008,7 @@ private void checkxEnabled() throws IgniteException { */ public List>> querySqlFields(final SqlFieldsQuery qry, final boolean keepBinary, final boolean failOnMultipleStmts) { - return querySqlFields(null, qry, keepBinary, failOnMultipleStmts); - } - - @SuppressWarnings("unchecked") - public FieldsQueryCursor> querySqlFields(final GridCacheContext cctx, final SqlFieldsQuery qry, - final boolean keepBinary) { - return querySqlFields(cctx, qry, keepBinary, true).get(0); + return querySqlFields(null, qry, null, keepBinary, failOnMultipleStmts); } /** @@ -1993,7 +2019,7 @@ public FieldsQueryCursor> querySqlFields(final GridCacheContext cct * @return Cursor. */ public FieldsQueryCursor> querySqlFields(final SqlFieldsQuery qry, final boolean keepBinary) { - return querySqlFields(null, qry, keepBinary, true).get(0); + return querySqlFields(null, qry, null, keepBinary, true).get(0); } /** @@ -2001,14 +2027,16 @@ public FieldsQueryCursor> querySqlFields(final SqlFieldsQuery qry, final * * @param cctx Cache context. * @param qry Query. + * @param cliCtx Client context. * @param keepBinary Keep binary flag. * @param failOnMultipleStmts If {@code true} the method must throws exception when query contains * more then one SQL statement. * @return Cursor. */ @SuppressWarnings("unchecked") - public List>> querySqlFields(@Nullable final GridCacheContext cctx, - final SqlFieldsQuery qry, final boolean keepBinary, final boolean failOnMultipleStmts) { + public List>> querySqlFields(@Nullable final GridCacheContext cctx, + final SqlFieldsQuery qry, final SqlClientContext cliCtx, final boolean keepBinary, + final boolean failOnMultipleStmts) { checkxEnabled(); validateSqlFieldsQuery(qry); @@ -2036,7 +2064,7 @@ public List>> querySqlFields(@Nullable final GridCache GridQueryCancel cancel = new GridQueryCancel(); List>> res = - idx.querySqlFields(schemaName, qry, keepBinary, failOnMultipleStmts, cancel); + idx.querySqlFields(schemaName, qry, cliCtx, keepBinary, failOnMultipleStmts, cancel); if (cctx != null) sendQueryExecutedEvent(qry.getSql(), qry.getArgs(), cctx.name()); @@ -2075,7 +2103,7 @@ private static void validateSqlFieldsQuery(SqlFieldsQuery qry) { * @param schemaName Schema name. * @param streamer Data streamer. * @param qry Query. - * @return Iterator. + * @return Update counter. */ public long streamUpdateQuery(@Nullable final String cacheName, final String schemaName, final IgniteDataStreamer streamer, final String qry, final Object[] args) { @@ -2101,6 +2129,33 @@ public long streamUpdateQuery(@Nullable final String cacheName, final String sch } } + /** + * @param schemaName Schema name. + * @param cliCtx Client context. + * @param qry Query. + * @param args Query arguments. + * @return Update counters. + */ + public List streamBatchedUpdateQuery(final String schemaName, final SqlClientContext cliCtx, + final String qry, final List args) { + if (!busyLock.enterBusy()) + throw new IllegalStateException("Failed to execute query (grid is stopping)."); + + try { + return executeQuery(GridCacheQueryType.SQL_FIELDS, qry, null, new IgniteOutClosureX>() { + @Override public List applyx() throws IgniteCheckedException { + return idx.streamBatchedUpdateQuery(schemaName, qry, args, cliCtx); + } + }, true); + } + catch (IgniteCheckedException e) { + throw new CacheException(e); + } + finally { + busyLock.leaveBusy(); + } + } + /** * Execute distributed SQL query. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/SqlClientContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/SqlClientContext.java new file mode 100644 index 0000000000000..b1855355cab01 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/SqlClientContext.java @@ -0,0 +1,195 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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 java.util.HashMap; +import java.util.Map; +import org.apache.ignite.IgniteDataStreamer; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.util.typedef.internal.U; + +/** + * Container for connection properties passed by various drivers (JDBC drivers, possibly ODBC) having notion of an + * SQL connection - Ignite basically does not have one.

+ * Also contains anything that a driver may need to share between threads processing queries of logically same client - + * see JDBC thin driver + */ +public class SqlClientContext implements AutoCloseable { + /** Kernal context. */ + private final GridKernalContext ctx; + + /** Distributed joins flag. */ + private final boolean distributedJoins; + + /** Enforce join order flag. */ + private final boolean enforceJoinOrder; + + /** Collocated flag. */ + private final boolean collocated; + + /** Lazy query execution flag. */ + private final boolean lazy; + + /** Skip reducer on update flag. */ + private final boolean skipReducerOnUpdate; + + /** Allow overwrites for duplicate keys on streamed {@code INSERT}s. */ + private final boolean streamAllowOverwrite; + + /** Parallel ops count per node for data streamer. */ + private final int streamNodeParOps; + + /** Node buffer size for data streamer. */ + private final int streamNodeBufSize; + + /** Auto flush frequency for streaming. */ + private final long streamFlushTimeout; + + /** Streamers for various caches. */ + private final Map> streamers; + + /** Logger. */ + private final IgniteLogger log; + + /** + * @param ctx Kernal context. + * @param distributedJoins Distributed joins flag. + * @param enforceJoinOrder Enforce join order flag. + * @param collocated Collocated flag. + * @param lazy Lazy query execution flag. + * @param skipReducerOnUpdate Skip reducer on update flag. + * @param stream Streaming state flag + * @param streamAllowOverwrite Allow overwrites for duplicate keys on streamed {@code INSERT}s. + * @param streamNodeParOps Parallel ops count per node for data streamer. + * @param streamNodeBufSize Node buffer size for data streamer. + * @param streamFlushTimeout Auto flush frequency for streaming. + */ + public SqlClientContext(GridKernalContext ctx, boolean distributedJoins, boolean enforceJoinOrder, + boolean collocated, boolean lazy, boolean skipReducerOnUpdate, boolean stream, boolean streamAllowOverwrite, + int streamNodeParOps, int streamNodeBufSize, long streamFlushTimeout) { + this.ctx = ctx; + this.distributedJoins = distributedJoins; + this.enforceJoinOrder = enforceJoinOrder; + this.collocated = collocated; + this.lazy = lazy; + this.skipReducerOnUpdate = skipReducerOnUpdate; + this.streamAllowOverwrite = streamAllowOverwrite; + this.streamNodeParOps = streamNodeParOps; + this.streamNodeBufSize = streamNodeBufSize; + this.streamFlushTimeout = streamFlushTimeout; + + streamers = stream ? new HashMap<>() : null; + + log = ctx.log(SqlClientContext.class.getName()); + + ctx.query().registerClientContext(this); + } + + /** + * @return Collocated flag. + */ + public boolean isCollocated() { + return collocated; + } + + /** + * @return Distributed joins flag. + */ + public boolean isDistributedJoins() { + return distributedJoins; + } + + /** + * @return Enforce join order flag. + */ + public boolean isEnforceJoinOrder() { + return enforceJoinOrder; + } + + /** + * @return Lazy query execution flag. + */ + public boolean isLazy() { + return lazy; + } + + /** + * @return Skip reducer on update flag, + */ + public boolean isSkipReducerOnUpdate() { + return skipReducerOnUpdate; + } + + /** + * @return Streaming state flag (on or off). + */ + public boolean isStream() { + return streamers != null; + } + + /** + * @param cacheName Cache name. + * @return Streamer for given cache. + */ + public IgniteDataStreamer streamerForCache(String cacheName) { + Map> curStreamers = streamers; + + if (curStreamers == null) + return null; + + IgniteDataStreamer res = curStreamers.get(cacheName); + + if (res != null) + return res; + + res = ctx.grid().dataStreamer(cacheName); + + IgniteDataStreamer exStreamer = curStreamers.putIfAbsent(cacheName, res); + + if (exStreamer == null) { + res.autoFlushFrequency(streamFlushTimeout); + + res.allowOverwrite(streamAllowOverwrite); + + if (streamNodeBufSize > 0) + res.perNodeBufferSize(streamNodeBufSize); + + if (streamNodeParOps > 0) + res.perNodeParallelOperations(streamNodeParOps); + + return res; + } + else { // Someone got ahead of us. + res.close(); + + return exStreamer; + } + } + + /** {@inheritDoc} */ + @Override public void close() throws Exception { + ctx.query().unregisterClientContext(this); + + if (streamers == null) + return; + + for (IgniteDataStreamer s : streamers.values()) + U.close(s, log); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlParser.java b/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlParser.java index 0627defee6ae5..70f72a7122bf8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlParser.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlParser.java @@ -122,7 +122,7 @@ private SqlCommand nextCommand0() { return cmd; } else - throw errorUnexpectedToken(lex, CREATE, DROP, COPY, ALTER); + throw errorUnexpectedToken(lex, CREATE, DROP, ALTER, COPY); case QUOTED: case MINUS: diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java index df27c5fa59ba5..6d7e9ae0cbdea 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java @@ -52,6 +52,7 @@ import org.apache.ignite.internal.processors.query.GridRunningQueryInfo; import org.apache.ignite.internal.processors.query.QueryField; import org.apache.ignite.internal.processors.query.QueryIndexDescriptorImpl; +import org.apache.ignite.internal.processors.query.SqlClientContext; import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitor; import org.apache.ignite.internal.util.GridSpinBusyLock; import org.apache.ignite.internal.util.lang.GridCloseableIterator; @@ -244,12 +245,18 @@ private static class FailedIndexing implements GridQueryIndexing { /** {@inheritDoc} */ @Override public List>> querySqlFields(String schemaName, SqlFieldsQuery qry, - boolean keepBinary, boolean failOnMultipleStmts, GridQueryCancel cancel) { + SqlClientContext cliCtx, boolean keepBinary, boolean failOnMultipleStmts, GridQueryCancel cancel) { return null; } /** {@inheritDoc} */ - @Override public long streamUpdateQuery(String spaceName, String qry, @Nullable Object[] params, + @Override public List streamBatchedUpdateQuery(String schemaName, String qry, List params, + SqlClientContext cliCtx) throws IgniteCheckedException { + return Collections.emptyList(); + } + + /** {@inheritDoc} */ + @Override public long streamUpdateQuery(String schemaName, String qry, @Nullable Object[] params, IgniteDataStreamer streamer) throws IgniteCheckedException { return 0; } @@ -372,8 +379,8 @@ private static class FailedIndexing implements GridQueryIndexing { } /** {@inheritDoc} */ - @Override public boolean isInsertStatement(PreparedStatement nativeStmt) { - return false; + @Override public void checkStatementStreamable(PreparedStatement nativeStmt) { + // No-op. } /** {@inheritDoc} */ 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 b11482880afe5..8d2a82025e601 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 @@ -299,7 +299,7 @@ public static Throwable assertThrows(@Nullable IgniteLogger log, Callable cal call.call(); } catch (Throwable e) { - if (cls != e.getClass()) { + if (cls != e.getClass() && !cls.isAssignableFrom(e.getClass())) { if (e.getClass() == CacheException.class && e.getCause() != null && e.getCause().getClass() == cls) e = e.getCause(); else { diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java index ea6c7c1a4a4e6..62dbd507b0880 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java @@ -43,10 +43,10 @@ import org.apache.ignite.cache.query.FieldsQueryCursor; import org.apache.ignite.cache.query.SqlFieldsQuery; import org.apache.ignite.internal.GridKernalContext; -import org.apache.ignite.internal.processors.bulkload.BulkLoadCacheWriter; -import org.apache.ignite.internal.processors.bulkload.BulkLoadProcessor; import org.apache.ignite.internal.processors.bulkload.BulkLoadAckClientParameters; +import org.apache.ignite.internal.processors.bulkload.BulkLoadCacheWriter; import org.apache.ignite.internal.processors.bulkload.BulkLoadParser; +import org.apache.ignite.internal.processors.bulkload.BulkLoadProcessor; import org.apache.ignite.internal.processors.bulkload.BulkLoadStreamerWriter; import org.apache.ignite.internal.processors.cache.CacheOperationContext; import org.apache.ignite.internal.processors.cache.GridCacheContext; @@ -385,6 +385,7 @@ GridQueryFieldsResult updateSqlFieldsLocal(String schemaName, Connection conn, P /** * Perform given statement against given data streamer. Only rows based INSERT is supported. * + * @param schemaName Schema name. * @param streamer Streamer to feed data to. * @param stmt Statement. * @param args Statement arguments. @@ -392,81 +393,74 @@ GridQueryFieldsResult updateSqlFieldsLocal(String schemaName, Connection conn, P * @throws IgniteCheckedException if failed. */ @SuppressWarnings({"unchecked", "ConstantConditions"}) - long streamUpdateQuery(IgniteDataStreamer streamer, PreparedStatement stmt, final Object[] args) + long streamUpdateQuery(String schemaName, IgniteDataStreamer streamer, PreparedStatement stmt, final Object[] args) throws IgniteCheckedException { + idx.checkStatementStreamable(stmt); + Prepared p = GridSqlQueryParser.prepared(stmt); assert p != null; - final UpdatePlan plan = UpdatePlanBuilder.planForStatement(p, true, idx, null, null, null); - - if (!F.eq(streamer.cacheName(), plan.cacheContext().name())) - throw new IgniteSQLException("Cross cache streaming is not supported, please specify cache explicitly" + - " in connection options", IgniteQueryErrorCode.UNSUPPORTED_OPERATION); - - if (plan.mode() == UpdateMode.INSERT && plan.rowCount() > 0) { - assert plan.isLocalSubquery(); + final UpdatePlan plan = getPlanForStatement(schemaName, null, p, null, true, null); - final GridCacheContext cctx = plan.cacheContext(); + assert plan.isLocalSubquery(); - QueryCursorImpl> cur; + final GridCacheContext cctx = plan.cacheContext(); - final ArrayList> data = new ArrayList<>(plan.rowCount()); + QueryCursorImpl> cur; - QueryCursorImpl> stepCur = new QueryCursorImpl<>(new Iterable>() { - @Override public Iterator> iterator() { - try { - Iterator> it; + final ArrayList> data = new ArrayList<>(plan.rowCount()); - if (!F.isEmpty(plan.selectQuery())) { - GridQueryFieldsResult res = idx.queryLocalSqlFields(idx.schema(cctx.name()), - plan.selectQuery(), F.asList(U.firstNotNull(args, X.EMPTY_OBJECT_ARRAY)), - null, false, 0, null); + QueryCursorImpl> stepCur = new QueryCursorImpl<>(new Iterable>() { + @Override public Iterator> iterator() { + try { + Iterator> it; - it = res.iterator(); - } - else - it = plan.createRows(U.firstNotNull(args, X.EMPTY_OBJECT_ARRAY)).iterator(); + if (!F.isEmpty(plan.selectQuery())) { + GridQueryFieldsResult res = idx.queryLocalSqlFields(idx.schema(cctx.name()), + plan.selectQuery(), F.asList(U.firstNotNull(args, X.EMPTY_OBJECT_ARRAY)), + null, false, 0, null); - return new GridQueryCacheObjectsIterator(it, idx.objectContext(), cctx.keepBinary()); - } - catch (IgniteCheckedException e) { - throw new IgniteException(e); + it = res.iterator(); } - } - }, null); - - data.addAll(stepCur.getAll()); + else + it = plan.createRows(U.firstNotNull(args, X.EMPTY_OBJECT_ARRAY)).iterator(); - cur = new QueryCursorImpl<>(new Iterable>() { - @Override public Iterator> iterator() { - return data.iterator(); + return new GridQueryCacheObjectsIterator(it, idx.objectContext(), cctx.keepBinary()); } - }, null); - - if (plan.rowCount() == 1) { - IgniteBiTuple t = plan.processRow(cur.iterator().next()); + catch (IgniteCheckedException e) { + throw new IgniteException(e); + } + } + }, null); - streamer.addData(t.getKey(), t.getValue()); + data.addAll(stepCur.getAll()); - return 1; + cur = new QueryCursorImpl<>(new Iterable>() { + @Override public Iterator> iterator() { + return data.iterator(); } + }, null); - Map rows = new LinkedHashMap<>(plan.rowCount()); + if (plan.rowCount() == 1) { + IgniteBiTuple t = plan.processRow(cur.iterator().next()); - for (List row : cur) { - final IgniteBiTuple t = plan.processRow(row); + streamer.addData(t.getKey(), t.getValue()); - rows.put(t.getKey(), t.getValue()); - } + return 1; + } + + Map rows = new LinkedHashMap<>(plan.rowCount()); - streamer.addData(rows); + for (List row : cur) { + final IgniteBiTuple t = plan.processRow(row); - return rows.size(); + rows.put(t.getKey(), t.getValue()); } - else - throw new IgniteSQLException("Only tuple based INSERT statements are supported in streaming mode", - IgniteQueryErrorCode.UNSUPPORTED_OPERATION); + + streamer.addData(rows); + + return rows.size(); } /** @@ -519,7 +513,7 @@ private UpdateResult executeUpdateStatement(String schemaName, final GridCacheCo .setPageSize(fieldsQry.getPageSize()) .setTimeout(fieldsQry.getTimeout(), TimeUnit.MILLISECONDS); - cur = (QueryCursorImpl>)idx.querySqlFields(schemaName, newFieldsQry, true, true, + cur = (QueryCursorImpl>)idx.querySqlFields(schemaName, newFieldsQry, null, true, true, cancel).get(0); } else if (plan.hasRows()) @@ -610,7 +604,7 @@ private UpdateResult processDmlSelectResult(GridCacheContext cctx, UpdatePlan pl * @return Update plan. */ @SuppressWarnings({"unchecked", "ConstantConditions"}) - private UpdatePlan getPlanForStatement(String schema, Connection conn, Prepared p, SqlFieldsQuery fieldsQry, + UpdatePlan getPlanForStatement(String schema, Connection conn, Prepared p, SqlFieldsQuery fieldsQry, boolean loc, @Nullable Integer errKeysPos) throws IgniteCheckedException { H2CachedStatementKey planKey = H2CachedStatementKey.forDmlStatement(schema, p.getSQL(), fieldsQry, 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 06c936b9867f6..50b148ba4492c 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 @@ -92,6 +92,7 @@ import org.apache.ignite.internal.processors.query.QueryField; import org.apache.ignite.internal.processors.query.QueryIndexDescriptorImpl; import org.apache.ignite.internal.processors.query.QueryUtils; +import org.apache.ignite.internal.processors.query.SqlClientContext; import org.apache.ignite.internal.processors.query.h2.database.H2RowFactory; import org.apache.ignite.internal.processors.query.h2.database.H2TreeIndex; import org.apache.ignite.internal.processors.query.h2.database.io.H2ExtrasInnerIO; @@ -100,6 +101,7 @@ import org.apache.ignite.internal.processors.query.h2.database.io.H2LeafIO; import org.apache.ignite.internal.processors.query.h2.ddl.DdlStatementsProcessor; import org.apache.ignite.internal.processors.query.h2.dml.DmlUtils; +import org.apache.ignite.internal.processors.query.h2.dml.UpdatePlan; import org.apache.ignite.internal.processors.query.h2.opt.GridH2DefaultTableEngine; import org.apache.ignite.internal.processors.query.h2.opt.GridH2IndexBase; import org.apache.ignite.internal.processors.query.h2.opt.GridH2PlainRowFactory; @@ -120,8 +122,8 @@ import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor; import org.apache.ignite.internal.sql.SqlParseException; import org.apache.ignite.internal.sql.SqlParser; -import org.apache.ignite.internal.sql.command.SqlBulkLoadCommand; import org.apache.ignite.internal.sql.command.SqlAlterTableCommand; +import org.apache.ignite.internal.sql.command.SqlBulkLoadCommand; import org.apache.ignite.internal.sql.command.SqlCommand; import org.apache.ignite.internal.sql.command.SqlCreateIndexCommand; import org.apache.ignite.internal.sql.command.SqlDropIndexCommand; @@ -149,7 +151,6 @@ import org.h2.api.ErrorCode; import org.h2.api.JavaObjectSerializer; import org.h2.command.Prepared; -import org.h2.command.dml.Insert; import org.h2.command.dml.NoOperation; import org.h2.engine.Session; import org.h2.engine.SysProperties; @@ -192,7 +193,7 @@ @SuppressWarnings({"UnnecessaryFullyQualifiedName", "NonFinalStaticVariableUsedInClassInitialization"}) public class IgniteH2Indexing implements GridQueryIndexing { public static final Pattern INTERNAL_CMD_RE = Pattern.compile( - "^(create|drop)\\s+index|^alter\\s+table|^copy", Pattern.CASE_INSENSITIVE); + "^(create|drop)\\s+index|^alter\\s+table|^copy|^set|^flush", Pattern.CASE_INSENSITIVE); /* * Register IO for indexes. @@ -501,10 +502,10 @@ private PreparedStatement prepare0(Connection c, String sql) throws SQLException } /** {@inheritDoc} */ - @Override public PreparedStatement prepareNativeStatement(String schemaName, String sql) throws SQLException { + @Override public PreparedStatement prepareNativeStatement(String schemaName, String sql) { Connection conn = connectionForSchema(schemaName); - return prepareStatement(conn, sql, true); + return prepareStatementAndCaches(conn, sql); } /** @@ -1014,7 +1015,60 @@ else if (DdlStatementsProcessor.isDdlStatement(p)) throw new IgniteSQLException(e); } - return dmlProc.streamUpdateQuery(streamer, stmt, params); + return dmlProc.streamUpdateQuery(schemaName, streamer, stmt, params); + } + + /** {@inheritDoc} */ + @SuppressWarnings("ForLoopReplaceableByForEach") + @Override public List streamBatchedUpdateQuery(String schemaName, String qry, List params, + SqlClientContext cliCtx) throws IgniteCheckedException { + if (cliCtx == null || !cliCtx.isStream()) { + U.warn(log, "Connection is not in streaming mode."); + + return zeroBatchedStreamedUpdateResult(params.size()); + } + + final Connection conn = connectionForSchema(schemaName); + + final PreparedStatement stmt = prepareStatementAndCaches(conn, qry); + + if (GridSqlQueryParser.checkMultipleStatements(stmt)) + throw new IgniteSQLException("Multiple statements queries are not supported for streaming mode.", + IgniteQueryErrorCode.UNSUPPORTED_OPERATION); + + checkStatementStreamable(stmt); + + Prepared p = GridSqlQueryParser.prepared(stmt); + + UpdatePlan plan = dmlProc.getPlanForStatement(schemaName, conn, p, null, true, null); + + IgniteDataStreamer streamer = cliCtx.streamerForCache(plan.cacheContext().name()); + + if (streamer != null) { + List res = new ArrayList<>(params.size()); + + for (int i = 0; i < params.size(); i++) + res.add(dmlProc.streamUpdateQuery(schemaName, streamer, stmt, params.get(i))); + + return res; + } + else { + U.warn(log, "Streaming has been turned off by concurrent command."); + + return zeroBatchedStreamedUpdateResult(params.size()); + } + } + + /** + * @param size Result size. + * @return List of given size filled with 0Ls. + */ + private static List zeroBatchedStreamedUpdateResult(int size) { + Long[] res = new Long[size]; + + Arrays.fill(res, 0); + + return Arrays.asList(res); } /** @@ -1400,7 +1454,7 @@ UpdateResult runDistributedUpdate( fqry.setTimeout(qry.getTimeout(), TimeUnit.MILLISECONDS); final QueryCursor> res = - querySqlFields(schemaName, fqry, keepBinary, true, null).get(0); + querySqlFields(schemaName, fqry, null, keepBinary, true, null).get(0); final Iterable> converted = new Iterable>() { @Override public Iterator> iterator() { @@ -1436,19 +1490,19 @@ UpdateResult runDistributedUpdate( * Try executing query using native facilities. * * @param schemaName Schema name. - * @param qry Query. + * @param sql Query. * @return Result or {@code null} if cannot parse/process this query. */ - private List>> tryQueryDistributedSqlFieldsNative(String schemaName, SqlFieldsQuery qry) { + private List>> tryQueryDistributedSqlFieldsNative(String schemaName, String sql) { // Heuristic check for fast return. - if (!INTERNAL_CMD_RE.matcher(qry.getSql().trim()).find()) + if (!INTERNAL_CMD_RE.matcher(sql.trim()).find()) return null; // Parse. SqlCommand cmd; try { - SqlParser parser = new SqlParser(schemaName, qry.getSql()); + SqlParser parser = new SqlParser(schemaName, sql); cmd = parser.nextCommand(); @@ -1456,15 +1510,20 @@ private List>> tryQueryDistributedSqlFieldsNative(Stri if (parser.nextCommand() != null) return null; - // Currently supported commands are: CREATE/DROP INDEX/COPY/ALTER TABLE + // Currently supported commands are: + // CREATE/DROP INDEX + // COPY + // ALTER TABLE + // SET STREAMING + // FLUSH STREAMER if (!(cmd instanceof SqlCreateIndexCommand || cmd instanceof SqlDropIndexCommand || - cmd instanceof SqlBulkLoadCommand || cmd instanceof SqlAlterTableCommand)) + cmd instanceof SqlAlterTableCommand || cmd instanceof SqlBulkLoadCommand)) return null; } catch (Exception e) { // Cannot parse, return. if (log.isDebugEnabled()) - log.debug("Failed to parse SQL with native parser [qry=" + qry.getSql() + ", err=" + e + ']'); + log.debug("Failed to parse SQL with native parser [qry=" + sql + ", err=" + e + ']'); if (!IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_SQL_PARSER_DISABLE_H2_FALLBACK)) return null; @@ -1474,24 +1533,24 @@ private List>> tryQueryDistributedSqlFieldsNative(Stri if (e instanceof SqlParseException) code = ((SqlParseException)e).code(); - throw new IgniteSQLException("Failed to parse DDL statement: " + qry.getSql() + ": " + e.getMessage(), + throw new IgniteSQLException("Failed to parse DDL statement: " + sql + ": " + e.getMessage(), code, e); } // Execute. if (cmd instanceof SqlBulkLoadCommand) { - FieldsQueryCursor> cursor = dmlProc.runNativeDmlStatement(qry.getSql(), cmd); + FieldsQueryCursor> cursor = dmlProc.runNativeDmlStatement(sql, cmd); return Collections.singletonList(cursor); } else { try { - FieldsQueryCursor> cursor = ddlProc.runDdlStatement(qry.getSql(), cmd); + FieldsQueryCursor> cursor = ddlProc.runDdlStatement(sql, cmd); return Collections.singletonList(cursor); } catch (IgniteCheckedException e) { - throw new IgniteSQLException("Failed to execute DDL statement [stmt=" + qry.getSql() + "]: " + throw new IgniteSQLException("Failed to execute DDL statement [stmt=" + sql + "]: " + e.getMessage(), e); } } @@ -1515,8 +1574,8 @@ private void checkQueryType(SqlFieldsQuery qry, boolean isQry) { /** {@inheritDoc} */ @SuppressWarnings("StringEquality") @Override public List>> querySqlFields(String schemaName, SqlFieldsQuery qry, - boolean keepBinary, boolean failOnMultipleStmts, GridQueryCancel cancel) { - List>> res = tryQueryDistributedSqlFieldsNative(schemaName, qry); + SqlClientContext cliCtx, boolean keepBinary, boolean failOnMultipleStmts, GridQueryCancel cancel) { + List>> res = tryQueryDistributedSqlFieldsNative(schemaName, qry.getSql()); if (res != null) return res; @@ -1554,8 +1613,8 @@ private void checkQueryType(SqlFieldsQuery qry, boolean isQry) { // We may use this cached statement only for local queries and non queries. if (qry.isLocal() || !prepared.isQuery()) - return (List>>)doRunPrepared(schemaName, prepared, qry, null, null, - keepBinary, cancel); + return (List>>)doRunPrepared(schemaName, prepared, qry, null, cliCtx, + null, keepBinary, cancel); } } @@ -1585,7 +1644,7 @@ private void checkQueryType(SqlFieldsQuery qry, boolean isQry) { firstArg += prepared.getParameters().size(); - res.addAll(doRunPrepared(schemaName, prepared, newQry, twoStepQry, meta, keepBinary, cancel)); + res.addAll(doRunPrepared(schemaName, prepared, newQry, twoStepQry, cliCtx, meta, keepBinary, cancel)); if (parseRes.twoStepQuery() != null && parseRes.twoStepQueryKey() != null && !parseRes.twoStepQuery().explain()) @@ -1601,14 +1660,14 @@ private void checkQueryType(SqlFieldsQuery qry, boolean isQry) { * @param prepared H2 command. * @param qry Fields query with flags. * @param twoStepQry Two-step query if this query must be executed in a distributed way. + * @param cliCtx Client context, or {@code null} if not applicable. * @param meta Metadata for {@code twoStepQry}. * @param keepBinary Whether binary objects must not be deserialized automatically. - * @param cancel Query cancel state holder. - * @return Query result. + * @param cancel Query cancel state holder. @return Query result. */ private List>> doRunPrepared(String schemaName, Prepared prepared, - SqlFieldsQuery qry, GridCacheTwoStepQuery twoStepQry, List meta, boolean keepBinary, - GridQueryCancel cancel) { + SqlFieldsQuery qry, GridCacheTwoStepQuery twoStepQry, @Nullable SqlClientContext cliCtx, + List meta, boolean keepBinary, GridQueryCancel cancel) { String sqlQry = qry.getSql(); boolean loc = qry.isLocal(); @@ -2277,10 +2336,10 @@ private Collection tables(String cacheName) { } /** {@inheritDoc} */ - @Override public boolean isInsertStatement(PreparedStatement nativeStmt) { - Prepared prep = GridSqlQueryParser.prepared(nativeStmt); - - return prep instanceof Insert; + @Override public void checkStatementStreamable(PreparedStatement nativeStmt) { + if (!GridSqlQueryParser.isStreamableInsertStatement(nativeStmt)) + throw new IgniteSQLException("Only tuple based INSERT statements are supported in streaming mode.", + IgniteQueryErrorCode.UNSUPPORTED_OPERATION); } /** {@inheritDoc} */ diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java index 6f5b51f1a3651..5441e369b1c8c 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java @@ -106,7 +106,7 @@ public void start(final GridKernalContext ctx, IgniteH2Indexing idx) { * @throws IgniteCheckedException On error. */ @SuppressWarnings("unchecked") - public FieldsQueryCursor> runDdlStatement(String sql, SqlCommand cmd) throws IgniteCheckedException{ + public FieldsQueryCursor> runDdlStatement(String sql, SqlCommand cmd) throws IgniteCheckedException { IgniteInternalFuture fut; try { @@ -211,12 +211,7 @@ else if (cmd instanceof SqlAlterTableCommand) { if (fut != null) fut.get(); - QueryCursorImpl> resCur = (QueryCursorImpl>)new QueryCursorImpl(Collections.singletonList - (Collections.singletonList(0L)), null, false); - - resCur.fieldsMeta(UPDATE_RESULT_META); - - return resCur; + return zeroCursor(); } catch (SchemaOperationException e) { throw convert(e); @@ -229,6 +224,19 @@ else if (cmd instanceof SqlAlterTableCommand) { } } + /** + * @return Single-column, single-row cursor with 0 as number of updated records. + */ + @SuppressWarnings("unchecked") + public static QueryCursorImpl> zeroCursor() { + QueryCursorImpl> resCur = (QueryCursorImpl>)new QueryCursorImpl(Collections.singletonList + (Collections.singletonList(0L)), null, false); + + resCur.fieldsMeta(UPDATE_RESULT_META); + + return resCur; + } + /** * Execute DDL statement. * diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java index 10d485a84c761..98fbb97eb80ad 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java @@ -505,7 +505,7 @@ public String selectQuery() { /** * @return Local subquery flag. */ - @Nullable public boolean isLocalSubquery() { + public boolean isLocalSubquery() { return isLocSubqry; } } diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java index bced83667aff4..d897ac7e9b84c 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java @@ -85,20 +85,21 @@ private UpdatePlanBuilder() { * @param loc Local query flag. * @param idx Indexing. * @param conn Connection. - * @param fieldsQuery Original query. + * @param fieldsQry Original query. * @return Update plan. */ public static UpdatePlan planForStatement(Prepared prepared, boolean loc, IgniteH2Indexing idx, - @Nullable Connection conn, @Nullable SqlFieldsQuery fieldsQuery, @Nullable Integer errKeysPos) + @Nullable Connection conn, @Nullable SqlFieldsQuery fieldsQry, @Nullable Integer errKeysPos) throws IgniteCheckedException { - assert !prepared.isQuery(); - GridSqlStatement stmt = new GridSqlQueryParser(false).parse(prepared); if (stmt instanceof GridSqlMerge || stmt instanceof GridSqlInsert) - return planForInsert(stmt, loc, idx, conn, fieldsQuery); + return planForInsert(stmt, loc, idx, conn, fieldsQry); + else if (stmt instanceof GridSqlUpdate || stmt instanceof GridSqlDelete) + return planForUpdate(stmt, loc, idx, conn, fieldsQry, errKeysPos); else - return planForUpdate(stmt, loc, idx, conn, fieldsQuery, errKeysPos); + throw new IgniteSQLException("Unsupported operation: " + prepared.getSQL(), + IgniteQueryErrorCode.UNSUPPORTED_OPERATION); } /** diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java index 04bc21226e37d..2d2c25ce643f0 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java @@ -1992,6 +1992,18 @@ private GridSqlElement parseExpression0(Expression expression, boolean calcTypes expression.getClass().getSimpleName() + ']'); } + /** + * Check if passed statement is insert statement eligible for streaming. + * + * @param nativeStmt Native statement. + * @return {@code True} if streamable insert. + */ + public static boolean isStreamableInsertStatement(PreparedStatement nativeStmt) { + Prepared prep = prepared(nativeStmt); + + return prep instanceof Insert && INSERT_QUERY.get((Insert)prep) == null; + } + /** * @param cond Condition. * @param o Object. 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 069bdd7092d16..cf8bb2ebe6b09 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 @@ -140,7 +140,7 @@ public void testTwoStepGroupAndAggregates() throws Exception { SqlFieldsQuery qry = new SqlFieldsQuery("select f.productId, p.name, f.price " + "from FactPurchase f, \"replicated-prod\".DimProduct p where p.id = f.productId "); - for (List o : qryProc.querySqlFields(cache.context(), qry, false, true).get(0).getAll()) { + for (List o : qryProc.querySqlFields(cache.context(), qry, null, false, true).get(0).getAll()) { X.println("___ -> " + o); set1.add((Integer)o.get(0)); @@ -154,7 +154,7 @@ public void testTwoStepGroupAndAggregates() throws Exception { qry = new SqlFieldsQuery("select productId from FactPurchase group by productId"); - for (List o : qryProc.querySqlFields(cache.context(), qry, false, true).get(0).getAll()) { + for (List o : qryProc.querySqlFields(cache.context(), qry, null, false, true).get(0).getAll()) { X.println("___ -> " + o); assertTrue(set0.add((Integer) o.get(0))); @@ -173,7 +173,7 @@ public void testTwoStepGroupAndAggregates() throws Exception { "where p.id = f.productId " + "group by f.productId, p.name"); - for (List o : qryProc.querySqlFields(cache.context(), qry, false, true).get(0).getAll()) { + for (List o : qryProc.querySqlFields(cache.context(), qry, null, false, true).get(0).getAll()) { X.println("___ -> " + o); assertTrue(names.add((String)o.get(0))); @@ -190,7 +190,7 @@ public void testTwoStepGroupAndAggregates() throws Exception { "group by f.productId, p.name " + "having s >= 15"); - for (List o : qryProc.querySqlFields(cache.context(), qry, false, true).get(0).getAll()) { + for (List o : qryProc.querySqlFields(cache.context(), qry, null, false, true).get(0).getAll()) { X.println("___ -> " + o); assertTrue(i(o, 1) >= 15); @@ -203,7 +203,7 @@ public void testTwoStepGroupAndAggregates() throws Exception { qry = new SqlFieldsQuery("select top 3 distinct productId " + "from FactPurchase f order by productId desc "); - for (List o : qryProc.querySqlFields(cache.context(), qry, false, true).get(0).getAll()) { + for (List o : qryProc.querySqlFields(cache.context(), qry, null, false, true).get(0).getAll()) { X.println("___ -> " + o); assertEquals(top--, o.get(0)); @@ -216,7 +216,7 @@ public void testTwoStepGroupAndAggregates() throws Exception { qry = new SqlFieldsQuery("select distinct productId " + "from FactPurchase f order by productId desc limit 2 offset 1"); - for (List o : qryProc.querySqlFields(cache.context(), qry, false, true).get(0).getAll()) { + for (List o : qryProc.querySqlFields(cache.context(), qry, null, false, true).get(0).getAll()) { X.println("___ -> " + o); assertEquals(top--, o.get(0)); @@ -256,13 +256,13 @@ public void testMultiStatement() throws Exception { GridTestUtils.assertThrows(log, new Callable() { @Override public Object call() throws Exception { - qryProc.querySqlFields(cache.context(), qry, false, true); + qryProc.querySqlFields(cache.context(), qry, null, false, true); return null; } }, IgniteSQLException.class, "Multiple statements queries are not supported"); - List>> cursors = qryProc.querySqlFields(cache.context(), qry, false, false); + List>> cursors = qryProc.querySqlFields(cache.context(), qry, null, false, false); assertEquals(2, cursors.size()); @@ -274,7 +274,7 @@ public void testMultiStatement() throws Exception { GridTestUtils.assertThrows(log, new Callable() { @Override public Object call() throws Exception { - qryProc.querySqlFields(cache.context(), qry, false, false); + qryProc.querySqlFields(cache.context(), qry, null, false, false); return null; } From 6707ac0792c63efd073dd6c26edb5c6a1f6b7c3f Mon Sep 17 00:00:00 2001 From: dpavlov Date: Tue, 13 Feb 2018 20:19:31 +0300 Subject: [PATCH 024/314] IGNITE-7695: Enable Ignite Update Notifier tests - Fixes #3516. Signed-off-by: Alexey Goncharuk (cherry-picked from commit e3f2259) --- .../java/org/apache/ignite/internal/GridVersionSelfTest.java | 2 -- .../internal/IgniteUpdateNotifierPerClusterSettingSelfTest.java | 2 -- 2 files changed, 4 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridVersionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridVersionSelfTest.java index 13af907489f4f..4751a0c4fb49b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/GridVersionSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/GridVersionSelfTest.java @@ -32,8 +32,6 @@ public class GridVersionSelfTest extends GridCommonAbstractTest { * @throws Exception If failed. */ public void testVersions() throws Exception { - fail("https://issues.apache.org/jira/browse/IGNITE-5413"); - String propVal = System.getProperty(IGNITE_UPDATE_NOTIFIER); System.setProperty(IGNITE_UPDATE_NOTIFIER, "true"); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteUpdateNotifierPerClusterSettingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteUpdateNotifierPerClusterSettingSelfTest.java index 5f11088194b58..a348ea555b4fe 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteUpdateNotifierPerClusterSettingSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteUpdateNotifierPerClusterSettingSelfTest.java @@ -66,8 +66,6 @@ public class IgniteUpdateNotifierPerClusterSettingSelfTest extends GridCommonAbs * @throws Exception If failed. */ public void testNotifierEnabledForCluster() throws Exception { - fail("https://issues.apache.org/jira/browse/IGNITE-5413"); - checkNotifierStatusForCluster(true); } From 9ba660cab2799f0606ff59997075f764b367b2aa Mon Sep 17 00:00:00 2001 From: devozerov Date: Mon, 19 Feb 2018 17:49:35 +0300 Subject: [PATCH 025/314] Release notes for COPY and streaming. --- RELEASE_NOTES.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/RELEASE_NOTES.txt b/RELEASE_NOTES.txt index 77db736468574..010b0df1eded0 100644 --- a/RELEASE_NOTES.txt +++ b/RELEASE_NOTES.txt @@ -32,6 +32,7 @@ Ignite.NET: SQL: * Added DEFAULT support to CREATE TABLE command * Added ALTER TABLE DROP COLUMN support +* Added COPY command for fast data load * Added INLINE_SIZE option to CREATE INDEX command * Added PARALLEL option to CREATE INDEX command * Added optional on-heap row cache @@ -46,6 +47,7 @@ SQL: JDBC Driver: * Optimized batched operations processing +* Added streaming mode to thin driver ODBC Driver: * Added support of multiple statements execution with one call From 9ef88099d844f7d32df747d6e3c6fd00c64dd684 Mon Sep 17 00:00:00 2001 From: Ilya Lantukh Date: Mon, 19 Feb 2018 18:06:56 +0300 Subject: [PATCH 026/314] IGNITE-7594 Fixed performance drop after WAL optimization for FSYNC mode by reverting the old implementation - Fixes #3521. Signed-off-by: Alexey Goncharuk --- .../apache/ignite/IgniteSystemProperties.java | 6 + .../DataStorageConfiguration.java | 11 +- .../PersistentStoreConfiguration.java | 7 +- .../apache/ignite/configuration/WALMode.java | 15 +- .../pagemem/wal/record/WALRecord.java | 19 +- .../processors/cache/GridCacheProcessor.java | 10 +- .../wal/AbstractWalRecordsIterator.java | 83 +- .../wal/FileWriteAheadLogManager.java | 107 +- .../FsyncModeFileWriteAheadLogManager.java | 3142 +++++++++++++++++ .../SingleSegmentLogicalRecordsIterator.java | 12 +- .../reader/StandaloneWalRecordsIterator.java | 13 +- .../db/wal/IgniteWalFlushDefaultSelfTest.java | 2 +- .../db/wal/reader/IgniteWalReaderTest.java | 4 +- 13 files changed, 3354 insertions(+), 77 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FsyncModeFileWriteAheadLogManager.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 43b718bb4b957..383d7003aba00 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -792,6 +792,12 @@ public final class IgniteSystemProperties { */ public static final String IGNITE_DIRECT_IO_ENABLED = "IGNITE_DIRECT_IO_ENABLED"; + /** + * When set to {@code true}, WAL implementation with dedicated worker will be used even in FSYNC mode. + * Default is {@code false}. + */ + public static final String IGNITE_WAL_FSYNC_WITH_DEDICATED_WORKER = "IGNITE_WAL_FSYNC_WITH_DEDICATED_WORKER"; + /** * Enforces singleton. diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java index 8d91503f9c127..4a3dbbf5a5b6a 100644 --- a/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java @@ -120,7 +120,7 @@ public class DataStorageConfiguration implements Serializable { public static final int DFLT_WAL_SEGMENT_SIZE = 64 * 1024 * 1024; /** Default wal mode. */ - public static final WALMode DFLT_WAL_MODE = WALMode.DEFAULT; + public static final WALMode DFLT_WAL_MODE = WALMode.LOG_ONLY; /** Default thread local buffer size. */ public static final int DFLT_TLB_SIZE = 128 * 1024; @@ -686,6 +686,9 @@ public WALMode getWalMode() { * @param walMode Wal mode. */ public DataStorageConfiguration setWalMode(WALMode walMode) { + if (walMode == WALMode.DEFAULT) + walMode = WALMode.FSYNC; + this.walMode = walMode; return this; @@ -755,7 +758,7 @@ public DataStorageConfiguration setWalFlushFrequency(long walFlushFreq) { } /** - * Property that allows to trade latency for throughput in {@link WALMode#DEFAULT} mode. + * Property that allows to trade latency for throughput in {@link WALMode#FSYNC} mode. * It limits minimum time interval between WAL fsyncs. First thread that initiates WAL fsync will wait for * this number of nanoseconds, another threads will just wait fsync of first thread (similar to CyclicBarrier). * Total throughput should increase under load as total WAL fsync rate will be limited. @@ -765,7 +768,7 @@ public long getWalFsyncDelayNanos() { } /** - * Sets property that allows to trade latency for throughput in {@link WALMode#DEFAULT} mode. + * Sets property that allows to trade latency for throughput in {@link WALMode#FSYNC} mode. * It limits minimum time interval between WAL fsyncs. First thread that initiates WAL fsync will wait for * this number of nanoseconds, another threads will just wait fsync of first thread (similar to CyclicBarrier). * Total throughput should increase under load as total WAL fsync rate will be limited. @@ -845,7 +848,7 @@ public DataStorageConfiguration setFileIOFactory(FileIOFactory fileIOFactory) { } /** - * Note: setting this value with {@link WALMode#DEFAULT} may generate file size overhead for WAL segments in case + * Note: setting this value with {@link WALMode#FSYNC} may generate file size overhead for WAL segments in case * grid is used rarely. * * @param walAutoArchiveAfterInactivity time in millis to run auto archiving segment (even if incomplete) after last diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java index d59d19b67b2d4..37a252844654f 100644 --- a/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java @@ -63,7 +63,7 @@ public class PersistentStoreConfiguration implements Serializable { public static final int DFLT_WAL_SEGMENT_SIZE = 64 * 1024 * 1024; /** Default wal mode. */ - public static final WALMode DFLT_WAL_MODE = WALMode.DEFAULT; + public static final WALMode DFLT_WAL_MODE = WALMode.LOG_ONLY; /** Default Wal flush frequency. */ public static final int DFLT_WAL_FLUSH_FREQ = 2000; @@ -483,6 +483,9 @@ public WALMode getWalMode() { * @param walMode Wal mode. */ public PersistentStoreConfiguration setWalMode(WALMode walMode) { + if (walMode == WALMode.DEFAULT) + walMode = WALMode.FSYNC; + this.walMode = walMode; return this; @@ -623,7 +626,7 @@ public PersistentStoreConfiguration setFileIOFactory(FileIOFactory fileIOFactory } /** - * Note: setting this value with {@link WALMode#DEFAULT} may generate file size overhead for WAL segments in case + * Note: setting this value with {@link WALMode#FSYNC} may generate file size overhead for WAL segments in case * grid is used rarely. * * @param walAutoArchiveAfterInactivity time in millis to run auto archiving segment (even if incomplete) after last diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/WALMode.java b/modules/core/src/main/java/org/apache/ignite/configuration/WALMode.java index aa5cfdd5d30ae..10471cff7845f 100644 --- a/modules/core/src/main/java/org/apache/ignite/configuration/WALMode.java +++ b/modules/core/src/main/java/org/apache/ignite/configuration/WALMode.java @@ -25,11 +25,11 @@ */ public enum WALMode { /** - * Default mode: full-sync disk writes. These writes survive power loss scenarios. When a control is returned + * FSYNC mode: full-sync disk writes. These writes survive power loss scenarios. When a control is returned * from the transaction commit operation, the changes are guaranteed to be persisted to disk according to the * transaction write synchronization mode. */ - DEFAULT, + FSYNC, /** * Log only mode: flushes application buffers. These writes survive process crash. When a control is returned @@ -48,7 +48,16 @@ public enum WALMode { * {@link Ignite#active(boolean)} method. If an Ignite node is terminated in NONE mode abruptly, it is likely * that the data stored on disk is corrupted and work directory will need to be cleared for a node restart. */ - NONE; + NONE, + + /** + * Default mode: full-sync disk writes. These writes survive power loss scenarios. When a control is returned + * from the transaction commit operation, the changes are guaranteed to be persisted to disk according to the + * transaction write synchronization mode. + * @deprecated This mode is no longer default and left here only for API compatibility. It is equivalent to the + * {@code FSYNC} mode. + */ + @Deprecated DEFAULT; /** * Enumerated values. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/WALRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/WALRecord.java index 8362a69ff9124..4fae179b63d28 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/WALRecord.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/WALRecord.java @@ -188,7 +188,7 @@ public static RecordType fromOrdinal(int ord) { /** * Fake record type, causes stop iterating and indicates segment EOF * Note: regular record type is incremented by 1 and minimal value written to file is also 1 - * For {@link WALMode#DEFAULT} this value is at least came from padding + * For {@link WALMode#FSYNC} this value is at least came from padding */ public static final int STOP_ITERATION_RECORD_TYPE = 0; } @@ -196,6 +196,9 @@ public static RecordType fromOrdinal(int ord) { /** */ private int size; + /** */ + private int chainSize; + /** */ @GridToStringExclude private WALRecord prev; @@ -203,6 +206,20 @@ public static RecordType fromOrdinal(int ord) { /** */ private WALPointer pos; + /** + * @param chainSize Chain size in bytes. + */ + public void chainSize(int chainSize) { + this.chainSize = chainSize; + } + + /** + * @return Get chain size in bytes. + */ + public int chainSize() { + return chainSize; + } + /** * @return Previous record in chain. */ 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 ec456e17e795d..94a9037b6a55a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java @@ -56,6 +56,7 @@ import org.apache.ignite.configuration.MemoryConfiguration; import org.apache.ignite.configuration.NearCacheConfiguration; import org.apache.ignite.configuration.TransactionConfiguration; +import org.apache.ignite.configuration.WALMode; import org.apache.ignite.events.EventType; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.IgniteClientDisconnectedCheckedException; @@ -98,6 +99,7 @@ import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteCacheSnapshotManager; import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotDiscoveryMessage; import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList; +import org.apache.ignite.internal.processors.cache.persistence.wal.FsyncModeFileWriteAheadLogManager; import org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager; import org.apache.ignite.internal.processors.cache.query.GridCacheDistributedQueryManager; import org.apache.ignite.internal.processors.cache.query.GridCacheLocalQueryManager; @@ -182,6 +184,9 @@ public class GridCacheProcessor extends GridProcessorAdapter { private final boolean startClientCaches = IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_START_CACHES_ON_JOIN, false); + private final boolean walFsyncWithDedicatedWorker = + IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_WAL_FSYNC_WITH_DEDICATED_WORKER, false); + /** Shared cache context. */ private GridCacheSharedContext sharedCtx; @@ -2360,7 +2365,10 @@ private GridCacheSharedContext createSharedContext(GridKernalContext kernalCtx, pageStoreMgr = new FilePageStoreManager(ctx); - walMgr = new FileWriteAheadLogManager(ctx); + if (ctx.config().getDataStorageConfiguration().getWalMode() == WALMode.FSYNC && !walFsyncWithDedicatedWorker) + walMgr = new FsyncModeFileWriteAheadLogManager(ctx); + else + walMgr = new FileWriteAheadLogManager(ctx); } else dbMgr = new IgniteCacheDatabaseSharedManager(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java index bf59c8166c8e7..65f3a20830109 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java @@ -31,6 +31,7 @@ import org.apache.ignite.internal.processors.cache.persistence.file.FileIO; import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory; import org.apache.ignite.internal.processors.cache.persistence.file.UnzipFileIO; +import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordSerializer; import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordSerializerFactory; import org.apache.ignite.internal.util.GridCloseableIteratorAdapter; import org.apache.ignite.internal.util.typedef.P2; @@ -62,7 +63,7 @@ public abstract class AbstractWalRecordsIterator /** * Current WAL segment read file handle. To be filled by subclass advanceSegment */ - private FileWriteAheadLogManager.ReadFileHandle currWalSegment; + private AbstractReadFileHandle currWalSegment; /** Logger */ @NotNull protected final IgniteLogger log; @@ -104,21 +105,6 @@ protected AbstractWalRecordsIterator( buf = new ByteBufferExpander(bufSize, ByteOrder.nativeOrder()); } - /** - * Scans provided folder for a WAL segment files - * @param walFilesDir directory to scan - * @return found WAL file descriptors - */ - protected static FileWriteAheadLogManager.FileDescriptor[] loadFileDescriptors(@NotNull final File walFilesDir) throws IgniteCheckedException { - final File[] files = walFilesDir.listFiles(FileWriteAheadLogManager.WAL_SEGMENT_COMPACTED_OR_RAW_FILE_FILTER); - - if (files == null) { - throw new IgniteCheckedException("WAL files directory does not not denote a " + - "directory, or if an I/O error occurs: [" + walFilesDir.getAbsolutePath() + "]"); - } - return FileWriteAheadLogManager.scan(files); - } - /** {@inheritDoc} */ @Override protected IgniteBiTuple onNext() throws IgniteCheckedException { IgniteBiTuple ret = curRec; @@ -188,8 +174,8 @@ protected void advance() throws IgniteCheckedException { * @return closed handle * @throws IgniteCheckedException if IO failed */ - @Nullable protected FileWriteAheadLogManager.ReadFileHandle closeCurrentWalSegment() throws IgniteCheckedException { - final FileWriteAheadLogManager.ReadFileHandle walSegmentClosed = currWalSegment; + @Nullable protected AbstractReadFileHandle closeCurrentWalSegment() throws IgniteCheckedException { + final AbstractReadFileHandle walSegmentClosed = currWalSegment; if (walSegmentClosed != null) { walSegmentClosed.close(); @@ -206,8 +192,8 @@ protected void advance() throws IgniteCheckedException { * @param curWalSegment current open WAL segment or null if there is no open segment yet * @return new WAL segment to read or null for stop iteration */ - protected abstract FileWriteAheadLogManager.ReadFileHandle advanceSegment( - @Nullable final FileWriteAheadLogManager.ReadFileHandle curWalSegment) throws IgniteCheckedException; + protected abstract AbstractReadFileHandle advanceSegment( + @Nullable final AbstractReadFileHandle curWalSegment) throws IgniteCheckedException; /** * Switches to new record @@ -215,15 +201,15 @@ protected abstract FileWriteAheadLogManager.ReadFileHandle advanceSegment( * @return next advanced record */ private IgniteBiTuple advanceRecord( - @Nullable final FileWriteAheadLogManager.ReadFileHandle hnd + @Nullable final AbstractReadFileHandle hnd ) throws IgniteCheckedException { if (hnd == null) return null; - FileWALPointer actualFilePtr = new FileWALPointer(hnd.idx, (int)hnd.in.position(), 0); + FileWALPointer actualFilePtr = new FileWALPointer(hnd.idx(), (int)hnd.in().position(), 0); try { - WALRecord rec = hnd.ser.readRecord(hnd.in, actualFilePtr); + WALRecord rec = hnd.ser().readRecord(hnd.in(), actualFilePtr); actualFilePtr.length(rec.size()); @@ -271,12 +257,12 @@ protected void handleRecordException( * @throws FileNotFoundException If segment file is missing. * @throws IgniteCheckedException If initialized failed due to another unexpected error. */ - protected FileWriteAheadLogManager.ReadFileHandle initReadHandle( - @NotNull final FileWriteAheadLogManager.FileDescriptor desc, + protected AbstractReadFileHandle initReadHandle( + @NotNull final AbstractFileDescriptor desc, @Nullable final FileWALPointer start) throws IgniteCheckedException, FileNotFoundException { try { - FileIO fileIO = desc.isCompressed() ? new UnzipFileIO(desc.file) : ioFactory.create(desc.file); + FileIO fileIO = desc.isCompressed() ? new UnzipFileIO(desc.file()) : ioFactory.create(desc.file()); try { IgniteBiTuple tup = FileWriteAheadLogManager.readSerializerVersionAndCompactedFlag(fileIO); @@ -290,7 +276,7 @@ protected FileWriteAheadLogManager.ReadFileHandle initReadHandle( FileInput in = new FileInput(fileIO, buf); - if (start != null && desc.idx == start.index()) { + if (start != null && desc.idx() == start.index()) { if (isCompacted) { if (start.fileOffset() != 0) serializerFactory.recordDeserializeFilter(new StartSeekingFilter(start)); @@ -303,8 +289,7 @@ protected FileWriteAheadLogManager.ReadFileHandle initReadHandle( } } - return new FileWriteAheadLogManager.ReadFileHandle( - fileIO, desc.idx, serializerFactory.createSerializer(serVer), in); + return createReadFileHandle(fileIO, desc.idx(), serializerFactory.createSerializer(serVer), in); } catch (SegmentEofException | EOFException ignore) { try { @@ -332,10 +317,18 @@ protected FileWriteAheadLogManager.ReadFileHandle initReadHandle( } catch (IOException e) { throw new IgniteCheckedException( - "Failed to initialize WAL segment: " + desc.file.getAbsolutePath(), e); + "Failed to initialize WAL segment: " + desc.file().getAbsolutePath(), e); } } + /** */ + protected abstract AbstractReadFileHandle createReadFileHandle( + FileIO fileIO, + long idx, + RecordSerializer ser, + FileInput in + ); + /** * Filter that drops all records until given start pointer is reached. */ @@ -364,4 +357,34 @@ private static class StartSeekingFilter implements P2 0) { int toWrite = Math.min(FILL_BUF.length, left); @@ -1708,7 +1708,7 @@ private SegmentArchiveResult archiveSegment(long absIdx) throws IgniteCheckedExc Files.move(dstTmpFile.toPath(), dstFile.toPath()); - if (mode == WALMode.DEFAULT) { + if (mode == WALMode.FSYNC) { try (FileIO f0 = ioFactory.create(dstFile, CREATE, READ, WRITE)) { f0.force(); } @@ -1884,7 +1884,7 @@ private void deleteObsoleteRawSegments() { Files.move(tmpZip.toPath(), zip.toPath()); - if (mode == WALMode.DEFAULT) { + if (mode == WALMode.FSYNC) { try (FileIO f0 = ioFactory.create(zip, CREATE, READ, WRITE)) { f0.force(); } @@ -2112,7 +2112,7 @@ private void checkFiles( if (checkFile.isDirectory()) throw new IgniteCheckedException("Failed to initialize WAL log segment (a directory with " + "the same name already exists): " + checkFile.getAbsolutePath()); - else if (checkFile.length() != dsCfg.getWalSegmentSize() && mode == WALMode.DEFAULT) + else if (checkFile.length() != dsCfg.getWalSegmentSize() && mode == WALMode.FSYNC) throw new IgniteCheckedException("Failed to initialize WAL log segment " + "(WAL segment size change is not supported in 'DEFAULT' WAL mode) " + "[filePath=" + checkFile.getAbsolutePath() + @@ -2223,7 +2223,7 @@ else if (hdrMagicNum == HeaderRecord.COMPACTED_MAGIC) /** * WAL file descriptor. */ - public static class FileDescriptor implements Comparable { + public static class FileDescriptor implements Comparable, AbstractWalRecordsIterator.AbstractFileDescriptor { /** */ protected final File file; @@ -2313,6 +2313,16 @@ public String getAbsolutePath() { public boolean isCompressed() { return file.getName().endsWith(".zip"); } + + /** {@inheritDoc} */ + @Override public File file() { + return file; + } + + /** {@inheritDoc} */ + @Override public long idx() { + return idx; + } } /** @@ -2338,7 +2348,7 @@ private FileHandle(FileIO fileIO, long idx) { /** * */ - public static class ReadFileHandle extends FileHandle { + public static class ReadFileHandle extends FileHandle implements AbstractWalRecordsIterator.AbstractReadFileHandle { /** Entry serializer. */ RecordSerializer ser; @@ -2357,7 +2367,7 @@ public static class ReadFileHandle extends FileHandle { * @param ser Entry serializer. * @param in File input. */ - ReadFileHandle( + public ReadFileHandle( FileIO fileIO, long idx, RecordSerializer ser, @@ -2380,6 +2390,26 @@ public void close() throws IgniteCheckedException { throw new IgniteCheckedException(e); } } + + /** {@inheritDoc} */ + @Override public long idx() { + return idx; + } + + /** {@inheritDoc} */ + @Override public FileInput in() { + return in; + } + + /** {@inheritDoc} */ + @Override public RecordSerializer ser() { + return ser; + } + + /** {@inheritDoc} */ + @Override public boolean workDir() { + return workDir; + } } /** @@ -2744,7 +2774,7 @@ private boolean close(boolean rollOver) throws IgniteCheckedException, StorageEx } // Do the final fsync. - if (mode == WALMode.DEFAULT) { + if (mode == WALMode.FSYNC) { if (mmap) ((MappedByteBuffer)buf.buf).force(); else @@ -2795,7 +2825,7 @@ private void signalNextAvailable() { lock.lock(); try { - assert cctx.kernalContext().invalidated() || written == lastFsyncPos || mode != WALMode.DEFAULT : + assert cctx.kernalContext().invalidated() || written == lastFsyncPos || mode != WALMode.FSYNC : "fsync [written=" + written + ", lastFsync=" + lastFsyncPos + ", idx=" + idx + ']'; fileIO = null; @@ -2914,22 +2944,22 @@ private RecordsIterator( /** {@inheritDoc} */ @Override protected ReadFileHandle initReadHandle( - @NotNull FileDescriptor desc, + @NotNull AbstractFileDescriptor desc, @Nullable FileWALPointer start ) throws IgniteCheckedException, FileNotFoundException { - if (decompressor != null && !desc.file.exists()) { + if (decompressor != null && !desc.file().exists()) { FileDescriptor zipFile = new FileDescriptor( - new File(walArchiveDir, FileDescriptor.fileName(desc.getIdx()) + ".zip")); + new File(walArchiveDir, FileDescriptor.fileName(desc.idx()) + ".zip")); if (!zipFile.file.exists()) { throw new FileNotFoundException("Both compressed and raw segment files are missing in archive " + - "[segmentIdx=" + desc.idx + "]"); + "[segmentIdx=" + desc.idx() + "]"); } - decompressor.decompressFile(desc.idx).get(); + decompressor.decompressFile(desc.idx()).get(); } - return super.initReadHandle(desc, start); + return (ReadFileHandle) super.initReadHandle(desc, start); } /** {@inheritDoc} */ @@ -2938,9 +2968,9 @@ private RecordsIterator( curRec = null; - final ReadFileHandle handle = closeCurrentWalSegment(); + final AbstractReadFileHandle handle = closeCurrentWalSegment(); - if (handle != null && handle.workDir) + if (handle != null && handle.workDir()) releaseWorkSegment(curWalSegmIdx); curWalSegmIdx = Integer.MAX_VALUE; @@ -2950,16 +2980,16 @@ private RecordsIterator( * @throws IgniteCheckedException If failed to initialize first file handle. */ private void init() throws IgniteCheckedException { - FileDescriptor[] descs = loadFileDescriptors(walArchiveDir); + AbstractFileDescriptor[] descs = loadFileDescriptors(walArchiveDir); if (start != null) { if (!F.isEmpty(descs)) { - if (descs[0].idx > start.index()) + if (descs[0].idx() > start.index()) throw new IgniteCheckedException("WAL history is too short " + "[descs=" + Arrays.asList(descs) + ", start=" + start + ']'); - for (FileDescriptor desc : descs) { - if (desc.idx == start.index()) { + for (AbstractFileDescriptor desc : descs) { + if (desc.idx() == start.index()) { curWalSegmIdx = start.index(); break; @@ -2967,7 +2997,7 @@ private void init() throws IgniteCheckedException { } if (curWalSegmIdx == -1) { - long lastArchived = descs[descs.length - 1].idx; + long lastArchived = descs[descs.length - 1].idx(); if (lastArchived > start.index()) throw new IgniteCheckedException("WAL history is corrupted (segment is missing): " + start); @@ -2984,7 +3014,7 @@ private void init() throws IgniteCheckedException { } } else - curWalSegmIdx = !F.isEmpty(descs) ? descs[0].idx : 0; + curWalSegmIdx = !F.isEmpty(descs) ? descs[0].idx() : 0; curWalSegmIdx--; @@ -2993,14 +3023,14 @@ private void init() throws IgniteCheckedException { } /** {@inheritDoc} */ - @Override protected ReadFileHandle advanceSegment( - @Nullable final ReadFileHandle curWalSegment + @Override protected AbstractReadFileHandle advanceSegment( + @Nullable final AbstractReadFileHandle curWalSegment ) throws IgniteCheckedException { if (curWalSegment != null) { curWalSegment.close(); - if (curWalSegment.workDir) - releaseWorkSegment(curWalSegment.idx); + if (curWalSegment.workDir()) + releaseWorkSegment(curWalSegment.idx()); } @@ -3068,6 +3098,12 @@ private void releaseWorkSegment(long absIdx) { if (archiver != null) archiver.releaseWorkSegment(absIdx); } + + /** {@inheritDoc} */ + @Override protected AbstractReadFileHandle createReadFileHandle(FileIO fileIO, long idx, + RecordSerializer ser, FileInput in) { + return new ReadFileHandle(fileIO, idx, ser, in); + } } /** @@ -3349,4 +3385,19 @@ private void writeBuffer(long pos, ByteBuffer buf) throws StorageException, Igni } } } + + /** + * Scans provided folder for a WAL segment files + * @param walFilesDir directory to scan + * @return found WAL file descriptors + */ + public static FileDescriptor[] loadFileDescriptors(@NotNull final File walFilesDir) throws IgniteCheckedException { + final File[] files = walFilesDir.listFiles(WAL_SEGMENT_COMPACTED_OR_RAW_FILE_FILTER); + + if (files == null) { + throw new IgniteCheckedException("WAL files directory does not not denote a " + + "directory, or if an I/O error occurs: [" + walFilesDir.getAbsolutePath() + "]"); + } + return scan(files); + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FsyncModeFileWriteAheadLogManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FsyncModeFileWriteAheadLogManager.java new file mode 100644 index 0000000000000..a4e98cf7d4564 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FsyncModeFileWriteAheadLogManager.java @@ -0,0 +1,3142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.persistence.wal; + +import java.io.BufferedInputStream; +import java.io.BufferedOutputStream; +import java.io.EOFException; +import java.io.File; +import java.io.FileFilter; +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.file.Files; +import java.sql.Time; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.NavigableMap; +import java.util.TreeMap; +import java.util.TreeSet; +import java.util.concurrent.PriorityBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; +import java.util.regex.Pattern; +import java.util.zip.ZipEntry; +import java.util.zip.ZipInputStream; +import java.util.zip.ZipOutputStream; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.IgniteSystemProperties; +import org.apache.ignite.configuration.DataStorageConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.configuration.WALMode; +import org.apache.ignite.events.EventType; +import org.apache.ignite.events.WalSegmentArchivedEvent; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.NodeInvalidator; +import org.apache.ignite.internal.managers.eventstorage.GridEventStorageManager; +import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager; +import org.apache.ignite.internal.pagemem.wal.StorageException; +import org.apache.ignite.internal.pagemem.wal.WALIterator; +import org.apache.ignite.internal.pagemem.wal.WALPointer; +import org.apache.ignite.internal.pagemem.wal.record.MarshalledRecord; +import org.apache.ignite.internal.pagemem.wal.record.SwitchSegmentRecord; +import org.apache.ignite.internal.pagemem.wal.record.WALRecord; +import org.apache.ignite.internal.processors.cache.CacheGroupContext; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter; +import org.apache.ignite.internal.processors.cache.persistence.DataStorageMetricsImpl; +import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager; +import org.apache.ignite.internal.processors.cache.persistence.file.FileIO; +import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory; +import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderSettings; +import org.apache.ignite.internal.processors.cache.persistence.wal.crc.PureJavaCrc32; +import org.apache.ignite.internal.processors.cache.persistence.wal.record.HeaderRecord; +import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordSerializer; +import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordSerializerFactory; +import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordSerializerFactoryImpl; +import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV1Serializer; +import org.apache.ignite.internal.processors.timeout.GridTimeoutObject; +import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor; +import org.apache.ignite.internal.util.GridUnsafe; +import org.apache.ignite.internal.util.future.GridFinishedFuture; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.typedef.CIX1; +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.apache.ignite.lang.IgniteBiTuple; +import org.apache.ignite.lang.IgnitePredicate; +import org.apache.ignite.lang.IgniteUuid; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import static java.nio.file.StandardOpenOption.CREATE; +import static java.nio.file.StandardOpenOption.READ; +import static java.nio.file.StandardOpenOption.WRITE; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_WAL_SERIALIZER_VERSION; + +/** + * File WAL manager. + */ +public class FsyncModeFileWriteAheadLogManager extends GridCacheSharedManagerAdapter implements IgniteWriteAheadLogManager { + /** */ + public static final FileDescriptor[] EMPTY_DESCRIPTORS = new FileDescriptor[0]; + + /** */ + public static final String WAL_SEGMENT_FILE_EXT = ".wal"; + + /** */ + private static final byte[] FILL_BUF = new byte[1024 * 1024]; + + /** Pattern for segment file names */ + private static final Pattern WAL_NAME_PATTERN = Pattern.compile("\\d{16}\\.wal"); + + /** */ + private static final Pattern WAL_TEMP_NAME_PATTERN = Pattern.compile("\\d{16}\\.wal\\.tmp"); + + /** WAL segment file filter, see {@link #WAL_NAME_PATTERN} */ + public static final FileFilter WAL_SEGMENT_FILE_FILTER = new FileFilter() { + @Override public boolean accept(File file) { + return !file.isDirectory() && WAL_NAME_PATTERN.matcher(file.getName()).matches(); + } + }; + + /** */ + private static final FileFilter WAL_SEGMENT_TEMP_FILE_FILTER = new FileFilter() { + @Override public boolean accept(File file) { + return !file.isDirectory() && WAL_TEMP_NAME_PATTERN.matcher(file.getName()).matches(); + } + }; + + /** */ + private static final Pattern WAL_SEGMENT_FILE_COMPACTED_PATTERN = Pattern.compile("\\d{16}\\.wal\\.zip"); + + /** WAL segment file filter, see {@link #WAL_NAME_PATTERN} */ + public static final FileFilter WAL_SEGMENT_COMPACTED_OR_RAW_FILE_FILTER = new FileFilter() { + @Override public boolean accept(File file) { + return !file.isDirectory() && (WAL_NAME_PATTERN.matcher(file.getName()).matches() || + WAL_SEGMENT_FILE_COMPACTED_PATTERN.matcher(file.getName()).matches()); + } + }; + + /** */ + private static final Pattern WAL_SEGMENT_TEMP_FILE_COMPACTED_PATTERN = Pattern.compile("\\d{16}\\.wal\\.zip\\.tmp"); + + /** */ + private static final FileFilter WAL_SEGMENT_FILE_COMPACTED_FILTER = new FileFilter() { + @Override public boolean accept(File file) { + return !file.isDirectory() && WAL_SEGMENT_FILE_COMPACTED_PATTERN.matcher(file.getName()).matches(); + } + }; + + /** */ + private static final FileFilter WAL_SEGMENT_TEMP_FILE_COMPACTED_FILTER = new FileFilter() { + @Override public boolean accept(File file) { + return !file.isDirectory() && WAL_SEGMENT_TEMP_FILE_COMPACTED_PATTERN.matcher(file.getName()).matches(); + } + }; + + /** Latest serializer version to use. */ + private static final int LATEST_SERIALIZER_VERSION = 2; + + /** */ + private final boolean alwaysWriteFullPages; + + /** WAL segment size in bytes */ + private final long maxWalSegmentSize; + + /** */ + private final WALMode mode; + + /** Thread local byte buffer size, see {@link #tlb} */ + private final int tlbSize; + + /** WAL flush frequency. Makes sense only for {@link WALMode#BACKGROUND} log WALMode. */ + private final long flushFreq; + + /** Fsync delay. */ + private final long fsyncDelay; + + /** */ + private final DataStorageConfiguration dsCfg; + + /** Events service */ + private final GridEventStorageManager evt; + + /** */ + private IgniteConfiguration igCfg; + + /** Persistence metrics tracker. */ + private DataStorageMetricsImpl metrics; + + /** */ + private File walWorkDir; + + /** WAL archive directory (including consistent ID as subfolder) */ + private File walArchiveDir; + + /** Serializer of latest version, used to read header record and for write records */ + private RecordSerializer serializer; + + /** Serializer latest version to use. */ + private final int serializerVersion = + IgniteSystemProperties.getInteger(IGNITE_WAL_SERIALIZER_VERSION, LATEST_SERIALIZER_VERSION); + + /** Latest segment cleared by {@link #truncate(WALPointer, WALPointer)}. */ + private volatile long lastTruncatedArchiveIdx = -1L; + + /** Factory to provide I/O interfaces for read/write operations with files */ + private final FileIOFactory ioFactory; + + /** Updater for {@link #currentHnd}, used for verify there are no concurrent update for current log segment handle */ + private static final AtomicReferenceFieldUpdater currentHndUpd = + AtomicReferenceFieldUpdater.newUpdater(FsyncModeFileWriteAheadLogManager.class, FileWriteHandle.class, "currentHnd"); + + /** + * Thread local byte buffer for saving serialized WAL records chain, see {@link FileWriteHandle#head}. + * Introduced to decrease number of buffers allocation. + * Used only for record itself is shorter than {@link #tlbSize}. + */ + private final ThreadLocal tlb = new ThreadLocal() { + @Override protected ByteBuffer initialValue() { + ByteBuffer buf = ByteBuffer.allocateDirect(tlbSize); + + buf.order(GridUnsafe.NATIVE_BYTE_ORDER); + + return buf; + } + }; + + /** */ + private volatile FileArchiver archiver; + + /** Compressor. */ + private volatile FileCompressor compressor; + + /** Decompressor. */ + private volatile FileDecompressor decompressor; + + /** */ + private final ThreadLocal lastWALPtr = new ThreadLocal<>(); + + /** Current log segment handle */ + private volatile FileWriteHandle currentHnd; + + /** Environment failure. */ + private volatile Throwable envFailed; + + /** + * Positive (non-0) value indicates WAL can be archived even if not complete
+ * See {@link DataStorageConfiguration#setWalAutoArchiveAfterInactivity(long)}
+ */ + private final long walAutoArchiveAfterInactivity; + + /** + * Container with last WAL record logged timestamp.
+ * Zero value means there was no records logged to current segment, skip possible archiving for this case
+ * Value is filled only for case {@link #walAutoArchiveAfterInactivity} > 0
+ */ + private AtomicLong lastRecordLoggedMs = new AtomicLong(); + + /** + * Cancellable task for {@link WALMode#BACKGROUND}, should be cancelled at shutdown + * Null for non background modes + */ + @Nullable private volatile GridTimeoutProcessor.CancelableTask backgroundFlushSchedule; + + /** + * Reference to the last added next archive timeout check object. + * Null if mode is not enabled. + * Should be cancelled at shutdown + */ + @Nullable private volatile GridTimeoutObject nextAutoArchiveTimeoutObj; + + /** + * @param ctx Kernal context. + */ + public FsyncModeFileWriteAheadLogManager(@NotNull final GridKernalContext ctx) { + igCfg = ctx.config(); + + DataStorageConfiguration dsCfg = igCfg.getDataStorageConfiguration(); + + assert dsCfg != null; + + this.dsCfg = dsCfg; + + maxWalSegmentSize = dsCfg.getWalSegmentSize(); + mode = dsCfg.getWalMode(); + tlbSize = dsCfg.getWalThreadLocalBufferSize(); + flushFreq = dsCfg.getWalFlushFrequency(); + fsyncDelay = dsCfg.getWalFsyncDelayNanos(); + alwaysWriteFullPages = dsCfg.isAlwaysWriteFullPages(); + ioFactory = dsCfg.getFileIOFactory(); + walAutoArchiveAfterInactivity = dsCfg.getWalAutoArchiveAfterInactivity(); + evt = ctx.event(); + + assert mode == WALMode.FSYNC : dsCfg; + } + + /** {@inheritDoc} */ + @Override public void start0() throws IgniteCheckedException { + if (!cctx.kernalContext().clientNode()) { + final PdsFolderSettings resolveFolders = cctx.kernalContext().pdsFolderResolver().resolveFolders(); + + checkWalConfiguration(); + + walWorkDir = initDirectory( + dsCfg.getWalPath(), + DataStorageConfiguration.DFLT_WAL_PATH, + resolveFolders.folderName(), + "write ahead log work directory" + ); + + walArchiveDir = initDirectory( + dsCfg.getWalArchivePath(), + DataStorageConfiguration.DFLT_WAL_ARCHIVE_PATH, + resolveFolders.folderName(), + "write ahead log archive directory" + ); + + serializer = new RecordSerializerFactoryImpl(cctx).createSerializer(serializerVersion); + + GridCacheDatabaseSharedManager dbMgr = (GridCacheDatabaseSharedManager)cctx.database(); + + metrics = dbMgr.persistentStoreMetricsImpl(); + + checkOrPrepareFiles(); + + IgniteBiTuple tup = scanMinMaxArchiveIndices(); + + lastTruncatedArchiveIdx = tup == null ? -1 : tup.get1() - 1; + + archiver = new FileArchiver(tup == null ? -1 : tup.get2()); + + if (dsCfg.isWalCompactionEnabled()) { + compressor = new FileCompressor(); + + decompressor = new FileDecompressor(); + } + + if (mode != WALMode.NONE) { + if (log.isInfoEnabled()) + log.info("Started write-ahead log manager [mode=" + mode + ']'); + } + else + U.quietAndWarn(log, "Started write-ahead log manager in NONE mode, persisted data may be lost in " + + "a case of unexpected node failure. Make sure to deactivate the cluster before shutdown."); + } + } + + /** + * @throws IgniteCheckedException if WAL store path is configured and archive path isn't (or vice versa) + */ + private void checkWalConfiguration() throws IgniteCheckedException { + if (dsCfg.getWalPath() == null ^ dsCfg.getWalArchivePath() == null) { + throw new IgniteCheckedException( + "Properties should be either both specified or both null " + + "[walStorePath = " + dsCfg.getWalPath() + + ", walArchivePath = " + dsCfg.getWalArchivePath() + "]" + ); + } + } + + /** {@inheritDoc} */ + @Override protected void stop0(boolean cancel) { + final GridTimeoutProcessor.CancelableTask schedule = backgroundFlushSchedule; + + if (schedule != null) + schedule.close(); + + final GridTimeoutObject timeoutObj = nextAutoArchiveTimeoutObj; + + if (timeoutObj != null) + cctx.time().removeTimeoutObject(timeoutObj); + + final FileWriteHandle currHnd = currentHandle(); + + try { + if (mode == WALMode.BACKGROUND) { + if (currHnd != null) + currHnd.flush((FileWALPointer)null, true); + } + + if (currHnd != null) + currHnd.close(false); + + if (archiver != null) + archiver.shutdown(); + + if (compressor != null) + compressor.shutdown(); + + if (decompressor != null) + decompressor.shutdown(); + } + catch (Exception e) { + U.error(log, "Failed to gracefully close WAL segment: " + currentHnd.fileIO, e); + } + } + + /** {@inheritDoc} */ + @Override public void onActivate(GridKernalContext kctx) throws IgniteCheckedException { + if (log.isDebugEnabled()) + log.debug("Activated file write ahead log manager [nodeId=" + cctx.localNodeId() + + " topVer=" + cctx.discovery().topologyVersionEx() + " ]"); + + start0(); + + if (!cctx.kernalContext().clientNode()) { + assert archiver != null; + archiver.start(); + + if (compressor != null) + compressor.start(); + + if (decompressor != null) + decompressor.start(); + } + } + + /** {@inheritDoc} */ + @Override public void onDeActivate(GridKernalContext kctx) { + if (log.isDebugEnabled()) + log.debug("DeActivate file write ahead log [nodeId=" + cctx.localNodeId() + + " topVer=" + cctx.discovery().topologyVersionEx() + " ]"); + + stop0(true); + + currentHnd = null; + } + + /** {@inheritDoc} */ + @Override public boolean isAlwaysWriteFullPages() { + return alwaysWriteFullPages; + } + + /** {@inheritDoc} */ + @Override public boolean isFullSync() { + return mode == WALMode.FSYNC; + } + + /** {@inheritDoc} */ + @Override public void resumeLogging(WALPointer lastPtr) throws IgniteCheckedException { + try { + assert currentHnd == null; + assert lastPtr == null || lastPtr instanceof FileWALPointer; + + FileWALPointer filePtr = (FileWALPointer)lastPtr; + + currentHnd = restoreWriteHandle(filePtr); + + if (currentHnd.serializer.version() != serializer.version()) { + if (log.isInfoEnabled()) + log.info("Record serializer version change detected, will start logging with a new WAL record " + + "serializer to a new WAL segment [curFile=" + currentHnd + ", newVer=" + serializer.version() + + ", oldVer=" + currentHnd.serializer.version() + ']'); + + rollOver(currentHnd); + } + + if (mode == WALMode.BACKGROUND) { + backgroundFlushSchedule = cctx.time().schedule(new Runnable() { + @Override public void run() { + doFlush(); + } + }, flushFreq, flushFreq); + } + + if (walAutoArchiveAfterInactivity > 0) + scheduleNextInactivityPeriodElapsedCheck(); + } + catch (StorageException e) { + throw new IgniteCheckedException(e); + } + } + + /** + * Schedules next check of inactivity period expired. Based on current record update timestamp. + * At timeout method does check of inactivity period and schedules new launch. + */ + private void scheduleNextInactivityPeriodElapsedCheck() { + final long lastRecMs = lastRecordLoggedMs.get(); + final long nextPossibleAutoArchive = (lastRecMs <= 0 ? U.currentTimeMillis() : lastRecMs) + walAutoArchiveAfterInactivity; + + if (log.isDebugEnabled()) + log.debug("Schedule WAL rollover check at " + new Time(nextPossibleAutoArchive).toString()); + + nextAutoArchiveTimeoutObj = new GridTimeoutObject() { + private final IgniteUuid id = IgniteUuid.randomUuid(); + + @Override public IgniteUuid timeoutId() { + return id; + } + + @Override public long endTime() { + return nextPossibleAutoArchive; + } + + @Override public void onTimeout() { + if (log.isDebugEnabled()) + log.debug("Checking if WAL rollover required (" + new Time(U.currentTimeMillis()).toString() + ")"); + + checkWalRolloverRequiredDuringInactivityPeriod(); + + scheduleNextInactivityPeriodElapsedCheck(); + } + }; + cctx.time().addTimeoutObject(nextAutoArchiveTimeoutObj); + } + + /** + * @return Latest serializer version. + */ + public int serializerVersion() { + return serializerVersion; + } + + /** + * Checks if there was elapsed significant period of inactivity. + * If WAL auto-archive is enabled using {@link #walAutoArchiveAfterInactivity} > 0 this method will activate + * roll over by timeout
+ */ + private void checkWalRolloverRequiredDuringInactivityPeriod() { + if (walAutoArchiveAfterInactivity <= 0) + return; // feature not configured, nothing to do + + final long lastRecMs = lastRecordLoggedMs.get(); + + if (lastRecMs == 0) + return; //no records were logged to current segment, does not consider inactivity + + final long elapsedMs = U.currentTimeMillis() - lastRecMs; + + if (elapsedMs <= walAutoArchiveAfterInactivity) + return; // not enough time elapsed since last write + + if (!lastRecordLoggedMs.compareAndSet(lastRecMs, 0)) + return; // record write occurred concurrently + + final FileWriteHandle handle = currentHandle(); + + try { + rollOver(handle); + } + catch (IgniteCheckedException e) { + U.error(log, "Unable to perform segment rollover: " + e.getMessage(), e); + + NodeInvalidator.INSTANCE.invalidate(cctx.kernalContext(), e); + } + } + + /** {@inheritDoc} */ + @SuppressWarnings("TooBroadScope") + @Override public WALPointer log(WALRecord record) throws IgniteCheckedException, StorageException { + if (serializer == null || mode == WALMode.NONE) + return null; + + FileWriteHandle currWrHandle = currentHandle(); + + // Logging was not resumed yet. + if (currWrHandle == null) + return null; + + // Need to calculate record size first. + record.size(serializer.size(record)); + + for (; ; currWrHandle = rollOver(currWrHandle)) { + WALPointer ptr = currWrHandle.addRecord(record); + + if (ptr != null) { + metrics.onWalRecordLogged(); + + lastWALPtr.set(ptr); + + if (walAutoArchiveAfterInactivity > 0) + lastRecordLoggedMs.set(U.currentTimeMillis()); + + return ptr; + } + + checkNode(); + + if (isStopping()) + throw new IgniteCheckedException("Stopping."); + } + } + + /** {@inheritDoc} */ + @Override public void fsync(WALPointer ptr) throws IgniteCheckedException, StorageException { + if (serializer == null || mode == WALMode.NONE) + return; + + FileWriteHandle cur = currentHandle(); + + // WAL manager was not started (client node). + if (cur == null) + return; + + FileWALPointer filePtr = (FileWALPointer)(ptr == null ? lastWALPtr.get() : ptr); + + // No need to sync if was rolled over. + if (filePtr != null && !cur.needFsync(filePtr)) + return; + + cur.fsync(filePtr, false); + } + + /** {@inheritDoc} */ + @Override public WALIterator replay(WALPointer start) + throws IgniteCheckedException, StorageException { + assert start == null || start instanceof FileWALPointer : "Invalid start pointer: " + start; + + FileWriteHandle hnd = currentHandle(); + + FileWALPointer end = null; + + if (hnd != null) + end = hnd.position(); + + return new RecordsIterator( + cctx, + walWorkDir, + walArchiveDir, + (FileWALPointer)start, + end, + dsCfg, + new RecordSerializerFactoryImpl(cctx), + ioFactory, + archiver, + decompressor, + log + ); + } + + /** {@inheritDoc} */ + @Override public boolean reserve(WALPointer start) throws IgniteCheckedException { + assert start != null && start instanceof FileWALPointer : "Invalid start pointer: " + start; + + if (mode == WALMode.NONE) + return false; + + FileArchiver archiver0 = archiver; + + if (archiver0 == null) + throw new IgniteCheckedException("Could not reserve WAL segment: archiver == null"); + + archiver0.reserve(((FileWALPointer)start).index()); + + if (!hasIndex(((FileWALPointer)start).index())) { + archiver0.release(((FileWALPointer)start).index()); + + return false; + } + + return true; + } + + /** {@inheritDoc} */ + @Override public void release(WALPointer start) throws IgniteCheckedException { + assert start != null && start instanceof FileWALPointer : "Invalid start pointer: " + start; + + if (mode == WALMode.NONE) + return; + + FileArchiver archiver0 = archiver; + + if (archiver0 == null) + throw new IgniteCheckedException("Could not release WAL segment: archiver == null"); + + archiver0.release(((FileWALPointer)start).index()); + } + + /** + * @param absIdx Absolulte index to check. + * @return {@code true} if has this index. + */ + private boolean hasIndex(long absIdx) { + String segmentName = FileDescriptor.fileName(absIdx); + + String zipSegmentName = FileDescriptor.fileName(absIdx) + ".zip"; + + boolean inArchive = new File(walArchiveDir, segmentName).exists() || + new File(walArchiveDir, zipSegmentName).exists(); + + if (inArchive) + return true; + + if (absIdx <= lastArchivedIndex()) + return false; + + FileWriteHandle cur = currentHnd; + + return cur != null && cur.idx >= absIdx; + } + + /** {@inheritDoc} */ + @Override public int truncate(WALPointer low, WALPointer high) { + if (high == null) + return 0; + + assert high instanceof FileWALPointer : high; + + // File pointer bound: older entries will be deleted from archive + FileWALPointer lowPtr = (FileWALPointer)low; + FileWALPointer highPtr = (FileWALPointer)high; + + FileDescriptor[] descs = scan(walArchiveDir.listFiles(WAL_SEGMENT_COMPACTED_OR_RAW_FILE_FILTER)); + + int deleted = 0; + + FileArchiver archiver0 = archiver; + + for (FileDescriptor desc : descs) { + if (lowPtr != null && desc.idx < lowPtr.index()) + continue; + + // Do not delete reserved or locked segment and any segment after it. + if (archiver0 != null && archiver0.reserved(desc.idx)) + return deleted; + + long lastArchived = archiver0 != null ? archiver0.lastArchivedAbsoluteIndex() : lastArchivedIndex(); + + // We need to leave at least one archived segment to correctly determine the archive index. + if (desc.idx < highPtr.index() && desc.idx < lastArchived) { + if (!desc.file.delete()) + U.warn(log, "Failed to remove obsolete WAL segment (make sure the process has enough rights): " + + desc.file.getAbsolutePath()); + else + deleted++; + + // Bump up the oldest archive segment index. + if (lastTruncatedArchiveIdx < desc.idx) + lastTruncatedArchiveIdx = desc.idx; + } + } + + return deleted; + } + + /** {@inheritDoc} */ + @Override public void allowCompressionUntil(WALPointer ptr) { + if (compressor != null) + compressor.allowCompressionUntil(((FileWALPointer)ptr).index()); + } + + /** {@inheritDoc} */ + @Override public int walArchiveSegments() { + long lastTruncated = lastTruncatedArchiveIdx; + + long lastArchived = archiver.lastArchivedAbsoluteIndex(); + + if (lastArchived == -1) + return 0; + + int res = (int)(lastArchived - lastTruncated); + + return res >= 0 ? res : 0; + } + + /** {@inheritDoc} */ + @Override public boolean reserved(WALPointer ptr) { + FileWALPointer fPtr = (FileWALPointer)ptr; + + FileArchiver archiver0 = archiver; + + return archiver0 != null && archiver0.reserved(fPtr.index()); + } + + /** {@inheritDoc} */ + @Override public boolean disabled(int grpId) { + CacheGroupContext ctx = cctx.cache().cacheGroup(grpId); + + return ctx != null && !ctx.walEnabled(); + } + + /** + * Lists files in archive directory and returns the index of last archived file. + * + * @return The absolute index of last archived file. + */ + private long lastArchivedIndex() { + long lastIdx = -1; + + for (File file : walArchiveDir.listFiles(WAL_SEGMENT_COMPACTED_OR_RAW_FILE_FILTER)) { + try { + long idx = Long.parseLong(file.getName().substring(0, 16)); + + lastIdx = Math.max(lastIdx, idx); + } + catch (NumberFormatException | IndexOutOfBoundsException ignore) { + + } + } + + return lastIdx; + } + + /** + * Lists files in archive directory and returns the indices of least and last archived files. + * In case of holes, first segment after last "hole" is considered as minimum. + * Example: minimum(0, 1, 10, 11, 20, 21, 22) should be 20 + * + * @return The absolute indices of min and max archived files. + */ + private IgniteBiTuple scanMinMaxArchiveIndices() { + TreeSet archiveIndices = new TreeSet<>(); + + for (File file : walArchiveDir.listFiles(WAL_SEGMENT_COMPACTED_OR_RAW_FILE_FILTER)) { + try { + long idx = Long.parseLong(file.getName().substring(0, 16)); + + archiveIndices.add(idx); + } + catch (NumberFormatException | IndexOutOfBoundsException ignore) { + // No-op. + } + } + + if (archiveIndices.isEmpty()) + return null; + else { + Long min = archiveIndices.first(); + Long max = archiveIndices.last(); + + if (max - min == archiveIndices.size() - 1) + return F.t(min, max); // Short path. + + for (Long idx : archiveIndices.descendingSet()) { + if (!archiveIndices.contains(idx - 1)) + return F.t(idx, max); + } + + throw new IllegalStateException("Should never happen if TreeSet is valid."); + } + } + + /** + * Creates a directory specified by the given arguments. + * + * @param cfg Configured directory path, may be {@code null}. + * @param defDir Default directory path, will be used if cfg is {@code null}. + * @param consId Local node consistent ID. + * @param msg File description to print out on successful initialization. + * @return Initialized directory. + * @throws IgniteCheckedException If failed to initialize directory. + */ + private File initDirectory(String cfg, String defDir, String consId, String msg) throws IgniteCheckedException { + File dir; + + if (cfg != null) { + File workDir0 = new File(cfg); + + dir = workDir0.isAbsolute() ? + new File(workDir0, consId) : + new File(U.resolveWorkDirectory(igCfg.getWorkDirectory(), cfg, false), consId); + } + else + dir = new File(U.resolveWorkDirectory(igCfg.getWorkDirectory(), defDir, false), consId); + + U.ensureDirectory(dir, msg, log); + + return dir; + } + + /** + * @return Current log segment handle. + */ + private FileWriteHandle currentHandle() { + return currentHnd; + } + + /** + * @param cur Handle that failed to fit the given entry. + * @return Handle that will fit the entry. + */ + private FileWriteHandle rollOver(FileWriteHandle cur) throws StorageException, IgniteCheckedException { + FileWriteHandle hnd = currentHandle(); + + if (hnd != cur) + return hnd; + + if (hnd.close(true)) { + FileWriteHandle next = initNextWriteHandle(cur.idx); + + boolean swapped = currentHndUpd.compareAndSet(this, hnd, next); + + assert swapped : "Concurrent updates on rollover are not allowed"; + + if (walAutoArchiveAfterInactivity > 0) + lastRecordLoggedMs.set(0); + + // Let other threads to proceed with new segment. + hnd.signalNextAvailable(); + } + else + hnd.awaitNext(); + + return currentHandle(); + } + + /** + * @param lastReadPtr Last read WAL file pointer. + * @return Initialized file write handle. + * @throws IgniteCheckedException If failed to initialize WAL write handle. + */ + private FileWriteHandle restoreWriteHandle(FileWALPointer lastReadPtr) throws IgniteCheckedException { + long absIdx = lastReadPtr == null ? 0 : lastReadPtr.index(); + + long segNo = absIdx % dsCfg.getWalSegments(); + + File curFile = new File(walWorkDir, FileDescriptor.fileName(segNo)); + + int offset = lastReadPtr == null ? 0 : lastReadPtr.fileOffset(); + int len = lastReadPtr == null ? 0 : lastReadPtr.length(); + + try { + FileIO fileIO = ioFactory.create(curFile); + + try { + int serVer = serializerVersion; + + // If we have existing segment, try to read version from it. + if (lastReadPtr != null) { + try { + serVer = readSerializerVersionAndCompactedFlag(fileIO).get1(); + } + catch (SegmentEofException | EOFException ignore) { + serVer = serializerVersion; + } + } + + RecordSerializer ser = new RecordSerializerFactoryImpl(cctx).createSerializer(serVer); + + if (log.isInfoEnabled()) + log.info("Resuming logging to WAL segment [file=" + curFile.getAbsolutePath() + + ", offset=" + offset + ", ver=" + serVer + ']'); + + FileWriteHandle hnd = new FileWriteHandle( + fileIO, + absIdx, + offset + len, + maxWalSegmentSize, + ser); + + // For new handle write serializer version to it. + if (lastReadPtr == null) + hnd.writeSerializerVersion(); + + archiver.currentWalIndex(absIdx); + + return hnd; + } + catch (IgniteCheckedException | IOException e) { + fileIO.close(); + + throw e; + } + } + catch (IOException e) { + throw new IgniteCheckedException("Failed to restore WAL write handle: " + curFile.getAbsolutePath(), e); + } + } + + /** + * Fills the file header for a new segment. + * Calling this method signals we are done with the segment and it can be archived. + * If we don't have prepared file yet and achiever is busy this method blocks + * + * @param curIdx current absolute segment released by WAL writer + * @return Initialized file handle. + * @throws StorageException If IO exception occurred. + * @throws IgniteCheckedException If failed. + */ + private FileWriteHandle initNextWriteHandle(long curIdx) throws StorageException, IgniteCheckedException { + try { + File nextFile = pollNextFile(curIdx); + + if (log.isDebugEnabled()) + log.debug("Switching to a new WAL segment: " + nextFile.getAbsolutePath()); + + FileIO fileIO = ioFactory.create(nextFile); + + FileWriteHandle hnd = new FileWriteHandle( + fileIO, + curIdx + 1, + 0, + maxWalSegmentSize, + serializer); + + hnd.writeSerializerVersion(); + + return hnd; + } + catch (IOException e) { + StorageException se = new StorageException("Unable to initialize WAL segment", e); + + NodeInvalidator.INSTANCE.invalidate(cctx.kernalContext(), se); + + throw se; + } + } + + /** + * Deletes temp files, creates and prepares new; Creates first segment if necessary + */ + private void checkOrPrepareFiles() throws IgniteCheckedException { + // Clean temp files. + { + File[] tmpFiles = walWorkDir.listFiles(WAL_SEGMENT_TEMP_FILE_FILTER); + + if (!F.isEmpty(tmpFiles)) { + for (File tmp : tmpFiles) { + boolean deleted = tmp.delete(); + + if (!deleted) + throw new IgniteCheckedException("Failed to delete previously created temp file " + + "(make sure Ignite process has enough rights): " + tmp.getAbsolutePath()); + } + } + } + + File[] allFiles = walWorkDir.listFiles(WAL_SEGMENT_FILE_FILTER); + + if (allFiles.length != 0 && allFiles.length > dsCfg.getWalSegments()) + throw new IgniteCheckedException("Failed to initialize wal (work directory contains " + + "incorrect number of segments) [cur=" + allFiles.length + ", expected=" + dsCfg.getWalSegments() + ']'); + + // Allocate the first segment synchronously. All other segments will be allocated by archiver in background. + if (allFiles.length == 0) { + File first = new File(walWorkDir, FileDescriptor.fileName(0)); + + createFile(first); + } + else + checkFiles(0, false, null); + } + + /** + * Clears the file with zeros. + * + * @param file File to format. + */ + private void formatFile(File file) throws IgniteCheckedException { + if (log.isDebugEnabled()) + log.debug("Formatting file [exists=" + file.exists() + ", file=" + file.getAbsolutePath() + ']'); + + try (FileIO fileIO = ioFactory.create(file, CREATE, READ, WRITE)) { + int left = dsCfg.getWalSegmentSize(); + + if (mode == WALMode.FSYNC) { + while (left > 0) { + int toWrite = Math.min(FILL_BUF.length, left); + + fileIO.write(FILL_BUF, 0, toWrite); + + left -= toWrite; + } + + fileIO.force(); + } + else + fileIO.clear(); + } + catch (IOException e) { + throw new IgniteCheckedException("Failed to format WAL segment file: " + file.getAbsolutePath(), e); + } + } + + /** + * Creates a file atomically with temp file. + * + * @param file File to create. + * @throws IgniteCheckedException If failed. + */ + private void createFile(File file) throws IgniteCheckedException { + if (log.isDebugEnabled()) + log.debug("Creating new file [exists=" + file.exists() + ", file=" + file.getAbsolutePath() + ']'); + + File tmp = new File(file.getParent(), file.getName() + ".tmp"); + + formatFile(tmp); + + try { + Files.move(tmp.toPath(), file.toPath()); + } + catch (IOException e) { + throw new IgniteCheckedException("Failed to move temp file to a regular WAL segment file: " + + file.getAbsolutePath(), e); + } + + if (log.isDebugEnabled()) + log.debug("Created WAL segment [file=" + file.getAbsolutePath() + ", size=" + file.length() + ']'); + } + + /** + * Retrieves next available file to write WAL data, waiting + * if necessary for a segment to become available. + * + * @param curIdx Current absolute WAL segment index. + * @return File ready for use as new WAL segment. + * @throws IgniteCheckedException If failed. + */ + private File pollNextFile(long curIdx) throws IgniteCheckedException { + // Signal to archiver that we are done with the segment and it can be archived. + long absNextIdx = archiver.nextAbsoluteSegmentIndex(curIdx); + + long segmentIdx = absNextIdx % dsCfg.getWalSegments(); + + return new File(walWorkDir, FileDescriptor.fileName(segmentIdx)); + } + + + /** + * @return Sorted WAL files descriptors. + */ + public static FileDescriptor[] scan(File[] allFiles) { + if (allFiles == null) + return EMPTY_DESCRIPTORS; + + FileDescriptor[] descs = new FileDescriptor[allFiles.length]; + + for (int i = 0; i < allFiles.length; i++) { + File f = allFiles[i]; + + descs[i] = new FileDescriptor(f); + } + + Arrays.sort(descs); + + return descs; + } + + /** + * @throws StorageException If node is no longer valid and we missed a WAL operation. + */ + private void checkNode() throws StorageException { + if (cctx.kernalContext().invalidated()) + throw new StorageException("Failed to perform WAL operation (environment was invalidated by a " + + "previous error)"); + } + + /** + * File archiver operates on absolute segment indexes. For any given absolute segment index N we can calculate + * the work WAL segment: S(N) = N % dsCfg.walSegments. + * When a work segment is finished, it is given to the archiver. If the absolute index of last archived segment + * is denoted by A and the absolute index of next segment we want to write is denoted by W, then we can allow + * write to S(W) if W - A <= walSegments.
+ * + * Monitor of current object is used for notify on: + *
    + *
  • exception occurred ({@link FileArchiver#cleanException}!=null)
  • + *
  • stopping thread ({@link FileArchiver#stopped}==true)
  • + *
  • current file index changed ({@link FileArchiver#curAbsWalIdx})
  • + *
  • last archived file index was changed ({@link FileArchiver#lastAbsArchivedIdx})
  • + *
  • some WAL index was removed from {@link FileArchiver#locked} map
  • + *
+ */ + private class FileArchiver extends Thread { + /** Exception which occurred during initial creation of files or during archiving WAL segment */ + private IgniteCheckedException cleanException; + + /** + * Absolute current segment index WAL Manager writes to. Guarded by this. + * Incremented during rollover. Also may be directly set if WAL is resuming logging after start. + */ + private long curAbsWalIdx = -1; + + /** Last archived file index (absolute, 0-based). Guarded by this. */ + private volatile long lastAbsArchivedIdx = -1; + + /** current thread stopping advice */ + private volatile boolean stopped; + + /** */ + private NavigableMap reserved = new TreeMap<>(); + + /** + * Maps absolute segment index to locks counter. Lock on segment protects from archiving segment and may + * come from {@link RecordsIterator} during WAL replay. Map itself is guarded by this. + */ + private Map locked = new HashMap<>(); + + /** + * + */ + private FileArchiver(long lastAbsArchivedIdx) { + super("wal-file-archiver%" + cctx.igniteInstanceName()); + + this.lastAbsArchivedIdx = lastAbsArchivedIdx; + } + + /** + * @return Last archived segment absolute index. + */ + private long lastArchivedAbsoluteIndex() { + return lastAbsArchivedIdx; + } + + /** + * @throws IgniteInterruptedCheckedException If failed to wait for thread shutdown. + */ + private void shutdown() throws IgniteInterruptedCheckedException { + synchronized (this) { + stopped = true; + + notifyAll(); + } + + U.join(this); + } + + /** + * @param curAbsWalIdx Current absolute WAL segment index. + */ + private void currentWalIndex(long curAbsWalIdx) { + synchronized (this) { + this.curAbsWalIdx = curAbsWalIdx; + + notifyAll(); + } + } + + /** + * @param absIdx Index for reservation. + */ + private synchronized void reserve(long absIdx) { + Integer cur = reserved.get(absIdx); + + if (cur == null) + reserved.put(absIdx, 1); + else + reserved.put(absIdx, cur + 1); + } + + /** + * Check if WAL segment locked or reserved + * + * @param absIdx Index for check reservation. + * @return {@code True} if index is reserved. + */ + private synchronized boolean reserved(long absIdx) { + return locked.containsKey(absIdx) || reserved.floorKey(absIdx) != null; + } + + /** + * @param absIdx Reserved index. + */ + private synchronized void release(long absIdx) { + Integer cur = reserved.get(absIdx); + + assert cur != null && cur >= 1 : cur; + + if (cur == 1) + reserved.remove(absIdx); + else + reserved.put(absIdx, cur - 1); + } + + /** {@inheritDoc} */ + @Override public void run() { + try { + allocateRemainingFiles(); + } + catch (IgniteCheckedException e) { + synchronized (this) { + // Stop the thread and report to starter. + cleanException = e; + + notifyAll(); + + return; + } + } + + try { + synchronized (this) { + while (curAbsWalIdx == -1 && !stopped) + wait(); + + if (curAbsWalIdx != 0 && lastAbsArchivedIdx == -1) + changeLastArchivedIndexAndWakeupCompressor(curAbsWalIdx - 1); + } + + while (!Thread.currentThread().isInterrupted() && !stopped) { + long toArchive; + + synchronized (this) { + assert lastAbsArchivedIdx <= curAbsWalIdx : "lastArchived=" + lastAbsArchivedIdx + + ", current=" + curAbsWalIdx; + + while (lastAbsArchivedIdx >= curAbsWalIdx - 1 && !stopped) + wait(); + + toArchive = lastAbsArchivedIdx + 1; + } + + if (stopped) + break; + + try { + final SegmentArchiveResult res = archiveSegment(toArchive); + + synchronized (this) { + while (locked.containsKey(toArchive) && !stopped) + wait(); + } + + // Firstly, format working file + if (!stopped) + formatFile(res.getOrigWorkFile()); + + synchronized (this) { + // Then increase counter to allow rollover on clean working file + changeLastArchivedIndexAndWakeupCompressor(toArchive); + + notifyAll(); + } + + if (evt.isRecordable(EventType.EVT_WAL_SEGMENT_ARCHIVED)) + evt.record(new WalSegmentArchivedEvent(cctx.discovery().localNode(), + res.getAbsIdx(), res.getDstArchiveFile())); + } + catch (IgniteCheckedException e) { + synchronized (this) { + cleanException = e; + + notifyAll(); + } + } + } + } + catch (InterruptedException ignore) { + Thread.currentThread().interrupt(); + } + } + + /** + * @param idx Index. + */ + private void changeLastArchivedIndexAndWakeupCompressor(long idx) { + lastAbsArchivedIdx = idx; + + if (compressor != null) + compressor.onNextSegmentArchived(); + } + + /** + * Gets the absolute index of the next WAL segment available to write. + * Blocks till there are available file to write + * + * @param curIdx Current absolute index that we want to increment. + * @return Next index (curWalSegmIdx+1) when it is ready to be written. + * @throws IgniteCheckedException If failed (if interrupted or if exception occurred in the archiver thread). + */ + private long nextAbsoluteSegmentIndex(long curIdx) throws IgniteCheckedException { + try { + synchronized (this) { + if (cleanException != null) + throw cleanException; + + assert curIdx == curAbsWalIdx; + + curAbsWalIdx++; + + // Notify archiver thread. + notifyAll(); + + while (curAbsWalIdx - lastAbsArchivedIdx > dsCfg.getWalSegments() && cleanException == null) + wait(); + + return curAbsWalIdx; + } + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + + throw new IgniteInterruptedCheckedException(e); + } + } + + /** + * @param absIdx Segment absolute index. + * @return
  • {@code True} if can read, no lock is held,
  • {@code false} if work segment, need + * release segment later, use {@link #releaseWorkSegment} for unlock
+ */ + @SuppressWarnings("NonPrivateFieldAccessedInSynchronizedContext") + private boolean checkCanReadArchiveOrReserveWorkSegment(long absIdx) { + synchronized (this) { + if (lastAbsArchivedIdx >= absIdx) { + if (log.isDebugEnabled()) + log.debug("Not needed to reserve WAL segment: absIdx=" + absIdx + ";" + + " lastAbsArchivedIdx=" + lastAbsArchivedIdx); + + return true; + } + + Integer cur = locked.get(absIdx); + + cur = cur == null ? 1 : cur + 1; + + locked.put(absIdx, cur); + + if (log.isDebugEnabled()) + log.debug("Reserved work segment [absIdx=" + absIdx + ", pins=" + cur + ']'); + + return false; + } + } + + /** + * @param absIdx Segment absolute index. + */ + @SuppressWarnings("NonPrivateFieldAccessedInSynchronizedContext") + private void releaseWorkSegment(long absIdx) { + synchronized (this) { + Integer cur = locked.get(absIdx); + + assert cur != null && cur > 0 : "WAL Segment with Index " + absIdx + " is not locked;" + + " lastAbsArchivedIdx = " + lastAbsArchivedIdx; + + if (cur == 1) { + locked.remove(absIdx); + + if (log.isDebugEnabled()) + log.debug("Fully released work segment (ready to archive) [absIdx=" + absIdx + ']'); + } + else { + locked.put(absIdx, cur - 1); + + if (log.isDebugEnabled()) + log.debug("Partially released work segment [absIdx=" + absIdx + ", pins=" + (cur - 1) + ']'); + } + + notifyAll(); + } + } + + /** + * Moves WAL segment from work folder to archive folder. + * Temp file is used to do movement + * + * @param absIdx Absolute index to archive. + */ + private SegmentArchiveResult archiveSegment(long absIdx) throws IgniteCheckedException { + long segIdx = absIdx % dsCfg.getWalSegments(); + + File origFile = new File(walWorkDir, FileDescriptor.fileName(segIdx)); + + String name = FileDescriptor.fileName(absIdx); + + File dstTmpFile = new File(walArchiveDir, name + ".tmp"); + + File dstFile = new File(walArchiveDir, name); + + if (log.isDebugEnabled()) + log.debug("Starting to copy WAL segment [absIdx=" + absIdx + ", segIdx=" + segIdx + + ", origFile=" + origFile.getAbsolutePath() + ", dstFile=" + dstFile.getAbsolutePath() + ']'); + + try { + Files.deleteIfExists(dstTmpFile.toPath()); + + Files.copy(origFile.toPath(), dstTmpFile.toPath()); + + Files.move(dstTmpFile.toPath(), dstFile.toPath()); + + if (mode == WALMode.FSYNC) { + try (FileIO f0 = ioFactory.create(dstFile, CREATE, READ, WRITE)) { + f0.force(); + } + } + } + catch (IOException e) { + throw new IgniteCheckedException("Failed to archive WAL segment [" + + "srcFile=" + origFile.getAbsolutePath() + + ", dstFile=" + dstTmpFile.getAbsolutePath() + ']', e); + } + + if (log.isDebugEnabled()) + log.debug("Copied file [src=" + origFile.getAbsolutePath() + + ", dst=" + dstFile.getAbsolutePath() + ']'); + + return new SegmentArchiveResult(absIdx, origFile, dstFile); + } + + /** + * + */ + private boolean checkStop() { + return stopped; + } + + /** + * Background creation of all segments except first. First segment was created in main thread by + * {@link FileWriteAheadLogManager#checkOrPrepareFiles()} + */ + private void allocateRemainingFiles() throws IgniteCheckedException { + checkFiles(1, true, new IgnitePredicate() { + @Override public boolean apply(Integer integer) { + return !checkStop(); + } + }); + } + } + + /** + * Responsible for compressing WAL archive segments. + * Also responsible for deleting raw copies of already compressed WAL archive segments if they are not reserved. + */ + private class FileCompressor extends Thread { + /** Current thread stopping advice. */ + private volatile boolean stopped; + + /** Last successfully compressed segment. */ + private volatile long lastCompressedIdx = -1L; + + /** All segments prior to this (inclusive) can be compressed. */ + private volatile long lastAllowedToCompressIdx = -1L; + + /** + * + */ + FileCompressor() { + super("wal-file-compressor%" + cctx.igniteInstanceName()); + } + + /** + * + */ + private void init() { + File[] toDel = walArchiveDir.listFiles(WAL_SEGMENT_TEMP_FILE_COMPACTED_FILTER); + + for (File f : toDel) { + if (stopped) + return; + + f.delete(); + } + + FileDescriptor[] alreadyCompressed = scan(walArchiveDir.listFiles(WAL_SEGMENT_FILE_COMPACTED_FILTER)); + + if (alreadyCompressed.length > 0) + lastCompressedIdx = alreadyCompressed[alreadyCompressed.length - 1].getIdx(); + } + + /** + * @param lastCpStartIdx Segment index to allow compression until (exclusively). + */ + synchronized void allowCompressionUntil(long lastCpStartIdx) { + lastAllowedToCompressIdx = lastCpStartIdx - 1; + + notify(); + } + + /** + * Callback for waking up compressor when new segment is archived. + */ + synchronized void onNextSegmentArchived() { + notify(); + } + + /** + * Pessimistically tries to reserve segment for compression in order to avoid concurrent truncation. + * Waits if there's no segment to archive right now. + */ + private long tryReserveNextSegmentOrWait() throws InterruptedException, IgniteCheckedException { + long segmentToCompress = lastCompressedIdx + 1; + + synchronized (this) { + while (segmentToCompress > Math.min(lastAllowedToCompressIdx, archiver.lastArchivedAbsoluteIndex())) { + wait(); + + if (stopped) + return -1; + } + } + + segmentToCompress = Math.max(segmentToCompress, lastTruncatedArchiveIdx + 1); + + boolean reserved = reserve(new FileWALPointer(segmentToCompress, 0, 0)); + + return reserved ? segmentToCompress : -1; + } + + /** + * + */ + private void deleteObsoleteRawSegments() { + FileDescriptor[] descs = scan(walArchiveDir.listFiles(WAL_SEGMENT_FILE_FILTER)); + + FileArchiver archiver0 = archiver; + + for (FileDescriptor desc : descs) { + // Do not delete reserved or locked segment and any segment after it. + if (archiver0 != null && archiver0.reserved(desc.idx)) + return; + + if (desc.idx < lastCompressedIdx) { + if (!desc.file.delete()) + U.warn(log, "Failed to remove obsolete WAL segment (make sure the process has enough rights): " + + desc.file.getAbsolutePath() + ", exists: " + desc.file.exists()); + } + } + } + + /** {@inheritDoc} */ + @Override public void run() { + init(); + + while (!Thread.currentThread().isInterrupted() && !stopped) { + try { + deleteObsoleteRawSegments(); + + long nextSegment = tryReserveNextSegmentOrWait(); + if (nextSegment == -1) + continue; + + File tmpZip = new File(walArchiveDir, FileDescriptor.fileName(nextSegment) + ".zip" + ".tmp"); + + File zip = new File(walArchiveDir, FileDescriptor.fileName(nextSegment) + ".zip"); + + File raw = new File(walArchiveDir, FileDescriptor.fileName(nextSegment)); + if (!Files.exists(raw.toPath())) + throw new IgniteCheckedException("WAL archive segment is missing: " + raw); + + compressSegmentToFile(nextSegment, raw, tmpZip); + + Files.move(tmpZip.toPath(), zip.toPath()); + + if (mode == WALMode.FSYNC) { + try (FileIO f0 = ioFactory.create(zip, CREATE, READ, WRITE)) { + f0.force(); + } + } + + lastCompressedIdx = nextSegment; + } + catch (IgniteCheckedException | IOException e) { + U.error(log, "Unexpected error during WAL compression", e); + + NodeInvalidator.INSTANCE.invalidate(cctx.kernalContext(), e); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + } + + /** + * @param nextSegment Next segment absolute idx. + * @param raw Raw file. + * @param zip Zip file. + */ + private void compressSegmentToFile(long nextSegment, File raw, File zip) + throws IOException, IgniteCheckedException { + int segmentSerializerVer; + + try (FileIO fileIO = ioFactory.create(raw)) { + IgniteBiTuple tup = FileWriteAheadLogManager.readSerializerVersionAndCompactedFlag(fileIO); + + segmentSerializerVer = tup.get1(); + } + + try (ZipOutputStream zos = new ZipOutputStream(new BufferedOutputStream(new FileOutputStream(zip)))) { + zos.putNextEntry(new ZipEntry("")); + + zos.write(prepareSerializerVersionBuffer(nextSegment, segmentSerializerVer, true).array()); + + final CIX1 appendToZipC = new CIX1() { + @Override public void applyx(WALRecord record) throws IgniteCheckedException { + final MarshalledRecord marshRec = (MarshalledRecord)record; + + try { + zos.write(marshRec.buffer().array(), 0, marshRec.buffer().remaining()); + } + catch (IOException e) { + throw new IgniteCheckedException(e); + } + } + }; + + try (SingleSegmentLogicalRecordsIterator iter = new SingleSegmentLogicalRecordsIterator( + log, cctx, ioFactory, tlbSize, nextSegment, walArchiveDir, appendToZipC)) { + + while (iter.hasNextX()) + iter.nextX(); + } + } + finally { + release(new FileWALPointer(nextSegment, 0, 0)); + } + } + + /** + * @throws IgniteInterruptedCheckedException If failed to wait for thread shutdown. + */ + private void shutdown() throws IgniteInterruptedCheckedException { + synchronized (this) { + stopped = true; + + notifyAll(); + } + + U.join(this); + } + } + + /** + * Responsible for decompressing previously compressed segments of WAL archive if they are needed for replay. + */ + private class FileDecompressor extends Thread { + /** Current thread stopping advice. */ + private volatile boolean stopped; + + /** Decompression futures. */ + private Map> decompressionFutures = new HashMap<>(); + + /** Segments queue. */ + private PriorityBlockingQueue segmentsQueue = new PriorityBlockingQueue<>(); + + /** Byte array for draining data. */ + private byte[] arr = new byte[tlbSize]; + + /** + * + */ + FileDecompressor() { + super("wal-file-decompressor%" + cctx.igniteInstanceName()); + } + + /** {@inheritDoc} */ + @Override public void run() { + while (!Thread.currentThread().isInterrupted() && !stopped) { + try { + long segmentToDecompress = segmentsQueue.take(); + + if (stopped) + break; + + File zip = new File(walArchiveDir, FileDescriptor.fileName(segmentToDecompress) + ".zip"); + File unzipTmp = new File(walArchiveDir, FileDescriptor.fileName(segmentToDecompress) + ".tmp"); + File unzip = new File(walArchiveDir, FileDescriptor.fileName(segmentToDecompress)); + + try (ZipInputStream zis = new ZipInputStream(new BufferedInputStream(new FileInputStream(zip))); + FileIO io = ioFactory.create(unzipTmp)) { + zis.getNextEntry(); + + int bytesRead; + while ((bytesRead = zis.read(arr)) > 0) + io.write(arr, 0, bytesRead); + } + + Files.move(unzipTmp.toPath(), unzip.toPath()); + + synchronized (this) { + decompressionFutures.remove(segmentToDecompress).onDone(); + } + } + catch (InterruptedException e){ + Thread.currentThread().interrupt(); + } + catch (IOException e) { + U.error(log, "Unexpected error during WAL decompression", e); + + NodeInvalidator.INSTANCE.invalidate(cctx.kernalContext(), e); + } + } + } + + /** + * Asynchronously decompresses WAL segment which is present only in .zip file. + * + * @return Future which is completed once file is decompressed. + */ + synchronized IgniteInternalFuture decompressFile(long idx) { + if (decompressionFutures.containsKey(idx)) + return decompressionFutures.get(idx); + + File f = new File(walArchiveDir, FileDescriptor.fileName(idx)); + + if (f.exists()) + return new GridFinishedFuture<>(); + + segmentsQueue.put(idx); + + GridFutureAdapter res = new GridFutureAdapter<>(); + + decompressionFutures.put(idx, res); + + return res; + } + + /** + * @throws IgniteInterruptedCheckedException If failed to wait for thread shutdown. + */ + private void shutdown() throws IgniteInterruptedCheckedException { + synchronized (this) { + stopped = true; + + // Put fake -1 to wake thread from queue.take() + segmentsQueue.put(-1L); + } + + U.join(this); + } + } + + /** + * Validate files depending on {@link DataStorageConfiguration#getWalSegments()} and create if need. + * Check end when exit condition return false or all files are passed. + * + * @param startWith Start with. + * @param create Flag create file. + * @param p Predicate Exit condition. + * @throws IgniteCheckedException if validation or create file fail. + */ + private void checkFiles(int startWith, boolean create, IgnitePredicate p) throws IgniteCheckedException { + for (int i = startWith; i < dsCfg.getWalSegments() && (p == null || (p != null && p.apply(i))); i++) { + File checkFile = new File(walWorkDir, FileDescriptor.fileName(i)); + + if (checkFile.exists()) { + if (checkFile.isDirectory()) + throw new IgniteCheckedException("Failed to initialize WAL log segment (a directory with " + + "the same name already exists): " + checkFile.getAbsolutePath()); + else if (checkFile.length() != dsCfg.getWalSegmentSize() && mode == WALMode.FSYNC) + throw new IgniteCheckedException("Failed to initialize WAL log segment " + + "(WAL segment size change is not supported):" + checkFile.getAbsolutePath()); + } + else if (create) + createFile(checkFile); + } + } + + /** + * Reads record serializer version from provided {@code io} along with compacted flag. + * NOTE: Method mutates position of {@code io}. + * + * @param io I/O interface for file. + * @return Serializer version stored in the file. + * @throws IgniteCheckedException If failed to read serializer version. + */ + public static IgniteBiTuple readSerializerVersionAndCompactedFlag(FileIO io) + throws IgniteCheckedException, IOException { + try (ByteBufferExpander buf = new ByteBufferExpander(RecordV1Serializer.HEADER_RECORD_SIZE, ByteOrder.nativeOrder())) { + FileInput in = new FileInput(io, buf); + + in.ensure(RecordV1Serializer.HEADER_RECORD_SIZE); + + int recordType = in.readUnsignedByte(); + + if (recordType == WALRecord.RecordType.STOP_ITERATION_RECORD_TYPE) + throw new SegmentEofException("Reached logical end of the segment", null); + + WALRecord.RecordType type = WALRecord.RecordType.fromOrdinal(recordType - 1); + + if (type != WALRecord.RecordType.HEADER_RECORD) + throw new IOException("Can't read serializer version", null); + + // Read file pointer. + FileWALPointer ptr = RecordV1Serializer.readPosition(in); + + assert ptr.fileOffset() == 0 : "Header record should be placed at the beginning of file " + ptr; + + long hdrMagicNum = in.readLong(); + + boolean compacted; + if (hdrMagicNum == HeaderRecord.REGULAR_MAGIC) + compacted = false; + else if (hdrMagicNum == HeaderRecord.COMPACTED_MAGIC) + compacted = true; + else { + throw new IOException("Magic is corrupted [exp=" + U.hexLong(HeaderRecord.REGULAR_MAGIC) + + ", actual=" + U.hexLong(hdrMagicNum) + ']'); + } + + // Read serializer version. + int ver = in.readInt(); + + // Read and skip CRC. + in.readInt(); + + return new IgniteBiTuple<>(ver, compacted); + } + } + + /** + * Writes record serializer version to provided {@code io}. + * NOTE: Method mutates position of {@code io}. + * + * @param io I/O interface for file. + * @param idx Segment index. + * @param version Serializer version. + * @return I/O position after write version. + * @throws IOException If failed to write serializer version. + */ + public static long writeSerializerVersion(FileIO io, long idx, int version, WALMode mode) throws IOException { + ByteBuffer buffer = prepareSerializerVersionBuffer(idx, version, false); + + do { + io.write(buffer); + } + while (buffer.hasRemaining()); + + // Flush + if (mode == WALMode.FSYNC) + io.force(); + + return io.position(); + } + + /** + * @param idx Index. + * @param ver Version. + * @param compacted Compacted flag. + */ + @NotNull private static ByteBuffer prepareSerializerVersionBuffer(long idx, int ver, boolean compacted) { + ByteBuffer buf = ByteBuffer.allocate(RecordV1Serializer.HEADER_RECORD_SIZE); + buf.order(ByteOrder.nativeOrder()); + + // Write record type. + buf.put((byte) (WALRecord.RecordType.HEADER_RECORD.ordinal() + 1)); + + // Write position. + RecordV1Serializer.putPosition(buf, new FileWALPointer(idx, 0, 0)); + + // Place magic number. + buf.putLong(compacted ? HeaderRecord.COMPACTED_MAGIC : HeaderRecord.REGULAR_MAGIC); + + // Place serializer version. + buf.putInt(ver); + + // Place CRC if needed. + if (!RecordV1Serializer.skipCrc) { + int curPos = buf.position(); + + buf.position(0); + + // This call will move buffer position to the end of the record again. + int crcVal = PureJavaCrc32.calcCrc32(buf, curPos); + + buf.putInt(crcVal); + } + else + buf.putInt(0); + + // Write header record through io. + buf.position(0); + + return buf; + } + + /** + * WAL file descriptor. + */ + public static class FileDescriptor implements Comparable, AbstractWalRecordsIterator.AbstractFileDescriptor { + /** */ + protected final File file; + + /** Absolute WAL segment file index */ + protected final long idx; + + /** + * Creates file descriptor. Index is restored from file name + * + * @param file WAL segment file. + */ + public FileDescriptor(@NotNull File file) { + this(file, null); + } + + /** + * @param file WAL segment file. + * @param idx Absolute WAL segment file index. For null value index is restored from file name + */ + public FileDescriptor(@NotNull File file, @Nullable Long idx) { + this.file = file; + + String fileName = file.getName(); + + assert fileName.contains(WAL_SEGMENT_FILE_EXT); + + this.idx = idx == null ? Long.parseLong(fileName.substring(0, 16)) : idx; + } + + /** + * @param segment Segment index. + * @return Segment file name. + */ + public static String fileName(long segment) { + SB b = new SB(); + + String segmentStr = Long.toString(segment); + + for (int i = segmentStr.length(); i < 16; i++) + b.a('0'); + + b.a(segmentStr).a(WAL_SEGMENT_FILE_EXT); + + return b.toString(); + } + + /** + * @param segment Segment number as integer. + * @return Segment number as aligned string. + */ + private static String segmentNumber(long segment) { + SB b = new SB(); + + String segmentStr = Long.toString(segment); + + for (int i = segmentStr.length(); i < 16; i++) + b.a('0'); + + b.a(segmentStr); + + return b.toString(); + } + + /** {@inheritDoc} */ + @Override public int compareTo(FileDescriptor o) { + return Long.compare(idx, o.idx); + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + + if (!(o instanceof FileDescriptor)) + return false; + + FileDescriptor that = (FileDescriptor)o; + + return idx == that.idx; + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return (int)(idx ^ (idx >>> 32)); + } + + /** + * @return Absolute WAL segment file index + */ + public long getIdx() { + return idx; + } + + /** + * @return absolute pathname string of this file descriptor pathname. + */ + public String getAbsolutePath() { + return file.getAbsolutePath(); + } + + /** {@inheritDoc} */ + @Override public boolean isCompressed() { + return file.getName().endsWith(".zip"); + } + + /** {@inheritDoc} */ + @Override public File file() { + return file; + } + + /** {@inheritDoc} */ + @Override public long idx() { + return idx; + } + } + + /** + * + */ + private abstract static class FileHandle { + /** I/O interface for read/write operations with file */ + protected FileIO fileIO; + + /** Absolute WAL segment file index (incremental counter) */ + protected final long idx; + + /** + * @param fileIO I/O interface for read/write operations of FileHandle. + * @param idx Absolute WAL segment file index (incremental counter). + */ + private FileHandle(FileIO fileIO, long idx) { + this.fileIO = fileIO; + this.idx = idx; + } + } + + /** + * + */ + public static class ReadFileHandle extends FileHandle implements AbstractWalRecordsIterator.AbstractReadFileHandle { + /** Entry serializer. */ + RecordSerializer ser; + + /** */ + FileInput in; + + /** + * true if this file handle came from work directory. + * false if this file handle came from archive directory. + */ + private boolean workDir; + + /** + * @param fileIO I/O interface for read/write operations of FileHandle. + * @param idx Absolute WAL segment file index (incremental counter). + * @param ser Entry serializer. + * @param in File input. + */ + ReadFileHandle( + FileIO fileIO, + long idx, + RecordSerializer ser, + FileInput in + ) { + super(fileIO, idx); + + this.ser = ser; + this.in = in; + } + + /** + * @throws IgniteCheckedException If failed to close the WAL segment file. + */ + public void close() throws IgniteCheckedException { + try { + fileIO.close(); + } + catch (IOException e) { + throw new IgniteCheckedException(e); + } + } + + /** {@inheritDoc} */ + @Override public long idx() { + return idx; + } + + /** {@inheritDoc} */ + @Override public FileInput in() { + return in; + } + + /** {@inheritDoc} */ + @Override public RecordSerializer ser() { + return ser; + } + + /** {@inheritDoc} */ + @Override public boolean workDir() { + return workDir; + } + } + + /** + * File handle for one log segment. + */ + @SuppressWarnings("SignalWithoutCorrespondingAwait") + private class FileWriteHandle extends FileHandle { + /** */ + private final RecordSerializer serializer; + + /** See {@link FileWriteAheadLogManager#maxWalSegmentSize} */ + private final long maxSegmentSize; + + /** + * Accumulated WAL records chain. + * This reference points to latest WAL record. + * When writing records chain is iterated from latest to oldest (see {@link WALRecord#previous()}) + * Records from chain are saved into buffer in reverse order + */ + private final AtomicReference head = new AtomicReference<>(); + + /** + * Position in current file after the end of last written record (incremented after file channel write + * operation) + */ + private volatile long written; + + /** */ + private volatile long lastFsyncPos; + + /** Stop guard to provide warranty that only one thread will be successful in calling {@link #close(boolean)}*/ + private final AtomicBoolean stop = new AtomicBoolean(false); + + /** */ + private final Lock lock = new ReentrantLock(); + + /** Condition activated each time writeBuffer() completes. Used to wait previously flushed write to complete */ + private final Condition writeComplete = lock.newCondition(); + + /** Condition for timed wait of several threads, see {@link DataStorageConfiguration#getWalFsyncDelayNanos()} */ + private final Condition fsync = lock.newCondition(); + + /** + * Next segment available condition. + * Protection from "spurious wakeup" is provided by predicate {@link #fileIO}=null + */ + private final Condition nextSegment = lock.newCondition(); + + /** + * @param fileIO I/O file interface to use + * @param idx Absolute WAL segment file index for easy access. + * @param pos Position. + * @param maxSegmentSize Max segment size. + * @param serializer Serializer. + * @throws IOException If failed. + */ + private FileWriteHandle( + FileIO fileIO, + long idx, + long pos, + long maxSegmentSize, + RecordSerializer serializer + ) throws IOException { + super(fileIO, idx); + + assert serializer != null; + + fileIO.position(pos); + + this.maxSegmentSize = maxSegmentSize; + this.serializer = serializer; + + head.set(new FakeRecord(new FileWALPointer(idx, (int)pos, 0), false)); + written = pos; + lastFsyncPos = pos; + } + + /** + * Write serializer version to current handle. + * NOTE: Method mutates {@code fileIO} position, written and lastFsyncPos fields. + * + * @throws IgniteCheckedException If fail to write serializer version. + */ + public void writeSerializerVersion() throws IgniteCheckedException { + try { + assert fileIO.position() == 0 : "Serializer version can be written only at the begin of file " + + fileIO.position(); + + long updatedPosition = FsyncModeFileWriteAheadLogManager.writeSerializerVersion(fileIO, idx, + serializer.version(), mode); + + written = updatedPosition; + lastFsyncPos = updatedPosition; + head.set(new FakeRecord(new FileWALPointer(idx, (int)updatedPosition, 0), false)); + } + catch (IOException e) { + throw new IgniteCheckedException("Unable to write serializer version for segment " + idx, e); + } + } + + /** + * Checks if current head is a close fake record and returns {@code true} if so. + * + * @return {@code true} if current head is close record. + */ + private boolean stopped() { + return stopped(head.get()); + } + + /** + * @param record Record to check. + * @return {@code true} if the record is fake close record. + */ + private boolean stopped(WALRecord record) { + return record instanceof FakeRecord && ((FakeRecord)record).stop; + } + + /** + * @param rec Record to be added to record chain as new {@link #head} + * @return Pointer or null if roll over to next segment is required or already started by other thread. + * @throws StorageException If failed. + * @throws IgniteCheckedException If failed. + */ + @Nullable private WALPointer addRecord(WALRecord rec) throws StorageException, IgniteCheckedException { + assert rec.size() > 0 || rec.getClass() == FakeRecord.class; + + boolean flushed = false; + + for (; ; ) { + WALRecord h = head.get(); + + long nextPos = nextPosition(h); + + if (nextPos + rec.size() >= maxSegmentSize || stopped(h)) { + // Can not write to this segment, need to switch to the next one. + return null; + } + + int newChainSize = h.chainSize() + rec.size(); + + if (newChainSize > tlbSize && !flushed) { + boolean res = h.previous() == null || flush(h, false); + + if (rec.size() > tlbSize) + flushed = res; + + continue; + } + + rec.chainSize(newChainSize); + rec.previous(h); + + FileWALPointer ptr = new FileWALPointer( + idx, + (int)nextPos, + rec.size()); + + rec.position(ptr); + + if (head.compareAndSet(h, rec)) + return ptr; + } + } + + /** + * @param rec Record. + * @return Position for the next record. + */ + private long nextPosition(WALRecord rec) { + return recordOffset(rec) + rec.size(); + } + + /** + * Flush or wait for concurrent flush completion. + * + * @param ptr Pointer. + * @throws IgniteCheckedException If failed. + */ + private void flushOrWait(FileWALPointer ptr, boolean stop) throws IgniteCheckedException { + long expWritten; + + if (ptr != null) { + // If requested obsolete file index, it must be already flushed by close. + if (ptr.index() != idx) + return; + + expWritten = ptr.fileOffset(); + } + else // We read head position before the flush because otherwise we can get wrong position. + expWritten = recordOffset(head.get()); + + if (flush(ptr, stop)) + return; + else if (stop) { + FakeRecord fr = (FakeRecord)head.get(); + + assert fr.stop : "Invalid fake record on top of the queue: " + fr; + + expWritten = recordOffset(fr); + } + + // Spin-wait for a while before acquiring the lock. + for (int i = 0; i < 64; i++) { + if (written >= expWritten) + return; + } + + // If we did not flush ourselves then await for concurrent flush to complete. + lock.lock(); + + try { + while (written < expWritten && envFailed == null) + U.awaitQuiet(writeComplete); + } + finally { + lock.unlock(); + } + } + + /** + * @param ptr Pointer. + * @return {@code true} If the flush really happened. + * @throws IgniteCheckedException If failed. + * @throws StorageException If failed. + */ + private boolean flush(FileWALPointer ptr, boolean stop) throws IgniteCheckedException, StorageException { + if (ptr == null) { // Unconditional flush. + for (; ; ) { + WALRecord expHead = head.get(); + + if (expHead.previous() == null) { + FakeRecord frHead = (FakeRecord)expHead; + + if (frHead.stop == stop || frHead.stop || + head.compareAndSet(expHead, new FakeRecord(frHead.position(), stop))) + return false; + } + + if (flush(expHead, stop)) + return true; + } + } + + assert ptr.index() == idx; + + for (; ; ) { + WALRecord h = head.get(); + + // If current chain begin position is greater than requested, then someone else flushed our changes. + if (chainBeginPosition(h) > ptr.fileOffset()) + return false; + + if (flush(h, stop)) + return true; // We are lucky. + } + } + + /** + * @param h Head of the chain. + * @return Chain begin position. + */ + private long chainBeginPosition(WALRecord h) { + return recordOffset(h) + h.size() - h.chainSize(); + } + + /** + * @param expHead Expected head of chain. If head was changed, flush is not performed in this thread + * @throws IgniteCheckedException If failed. + * @throws StorageException If failed. + */ + private boolean flush(WALRecord expHead, boolean stop) throws StorageException, IgniteCheckedException { + if (expHead.previous() == null) { + FakeRecord frHead = (FakeRecord)expHead; + + if (!stop || frHead.stop) // Protects from CASing terminal FakeRecord(true) to FakeRecord(false) + return false; + } + + // Fail-fast before CAS. + checkNode(); + + if (!head.compareAndSet(expHead, new FakeRecord(new FileWALPointer(idx, (int)nextPosition(expHead), 0), stop))) + return false; + + if (expHead.chainSize() == 0) + return false; + + // At this point we grabbed the piece of WAL chain. + // Any failure in this code must invalidate the environment. + try { + // We can safely allow other threads to start building next chains while we are doing flush here. + ByteBuffer buf; + + boolean tmpBuf = false; + + if (expHead.chainSize() > tlbSize) { + buf = GridUnsafe.allocateBuffer(expHead.chainSize()); + + tmpBuf = true; // We need to manually release this temporary direct buffer. + } + else + buf = tlb.get(); + + try { + long pos = fillBuffer(buf, expHead); + + writeBuffer(pos, buf); + } + finally { + if (tmpBuf) + GridUnsafe.freeBuffer(buf); + } + + return true; + } + catch (Throwable e) { + StorageException se = new StorageException("Unable to write", new IOException(e)); + + NodeInvalidator.INSTANCE.invalidate(cctx.kernalContext(), se); + + // All workers waiting for a next segment must be woken up and stopped + signalNextAvailable(); + + throw se; + } + } + + /** + * Serializes WAL records chain to provided byte buffer + * + * @param buf Buffer, will be filled with records chain from end to beginning + * @param head Head of the chain to write to the buffer. + * @return Position in file for this buffer. + * @throws IgniteCheckedException If failed. + */ + private long fillBuffer(ByteBuffer buf, WALRecord head) throws IgniteCheckedException { + final int limit = head.chainSize(); + + assert limit <= buf.capacity(); + + buf.rewind(); + buf.limit(limit); + + do { + buf.position(head.chainSize() - head.size()); + buf.limit(head.chainSize()); // Just to make sure that serializer works in bounds. + + try { + serializer.writeRecord(head, buf); + } + catch (RuntimeException e) { + throw new IllegalStateException("Failed to write record: " + head, e); + } + + assert !buf.hasRemaining() : "Reported record size is greater than actual: " + head; + + head = head.previous(); + } + while (head.previous() != null); + + assert head instanceof FakeRecord : head.getClass(); + + buf.rewind(); + buf.limit(limit); + + return recordOffset(head); + } + + /** + * Non-blocking check if this pointer needs to be sync'ed. + * + * @param ptr WAL pointer to check. + * @return {@code False} if this pointer has been already sync'ed. + */ + private boolean needFsync(FileWALPointer ptr) { + // If index has changed, it means that the log was rolled over and already sync'ed. + // If requested position is smaller than last sync'ed, it also means all is good. + // If position is equal, then our record is the last not synced. + return idx == ptr.index() && lastFsyncPos <= ptr.fileOffset(); + } + + /** + * @return Pointer to the end of the last written record (probably not fsync-ed). + */ + private FileWALPointer position() { + lock.lock(); + + try { + return new FileWALPointer(idx, (int)written, 0); + } + finally { + lock.unlock(); + } + } + + /** + * @param ptr Pointer to sync. + * @throws StorageException If failed. + */ + private void fsync(FileWALPointer ptr, boolean stop) throws StorageException, IgniteCheckedException { + lock.lock(); + + try { + if (ptr != null) { + if (!needFsync(ptr)) + return; + + if (fsyncDelay > 0 && !stopped()) { + // Delay fsync to collect as many updates as possible: trade latency for throughput. + U.await(fsync, fsyncDelay, TimeUnit.NANOSECONDS); + + if (!needFsync(ptr)) + return; + } + } + + flushOrWait(ptr, stop); + + if (stopped()) + return; + + if (lastFsyncPos != written) { + assert lastFsyncPos < written; // Fsync position must be behind. + + boolean metricsEnabled = metrics.metricsEnabled(); + + long start = metricsEnabled ? System.nanoTime() : 0; + + try { + fileIO.force(); + } + catch (IOException e) { + throw new StorageException(e); + } + + lastFsyncPos = written; + + if (fsyncDelay > 0) + fsync.signalAll(); + + long end = metricsEnabled ? System.nanoTime() : 0; + + if (metricsEnabled) + metrics.onFsync(end - start); + } + } + finally { + lock.unlock(); + } + } + + /** + * @return {@code true} If this thread actually closed the segment. + * @throws IgniteCheckedException If failed. + * @throws StorageException If failed. + */ + private boolean close(boolean rollOver) throws IgniteCheckedException, StorageException { + if (stop.compareAndSet(false, true)) { + lock.lock(); + + try { + flushOrWait(null, true); + + assert stopped() : "Segment is not closed after close flush: " + head.get(); + + try { + RecordSerializer backwardSerializer = new RecordSerializerFactoryImpl(cctx) + .createSerializer(serializerVersion); + + SwitchSegmentRecord segmentRecord = new SwitchSegmentRecord(); + + int switchSegmentRecSize = backwardSerializer.size(segmentRecord); + + if (rollOver && written < (maxSegmentSize - switchSegmentRecSize)) { + final ByteBuffer buf = ByteBuffer.allocate(switchSegmentRecSize); + + segmentRecord.position(new FileWALPointer(idx, (int)written, switchSegmentRecSize)); + backwardSerializer.writeRecord(segmentRecord, buf); + + buf.rewind(); + + int rem = buf.remaining(); + + while (rem > 0) { + int written0 = fileIO.write(buf, written); + + written += written0; + + rem -= written0; + } + } + + // Do the final fsync. + if (mode == WALMode.FSYNC) { + fileIO.force(); + + lastFsyncPos = written; + } + + fileIO.close(); + } + catch (IOException e) { + throw new IgniteCheckedException(e); + } + + if (log.isDebugEnabled()) + log.debug("Closed WAL write handle [idx=" + idx + "]"); + + return true; + } + finally { + lock.unlock(); + } + } + else + return false; + } + + /** + * Signals next segment available to wake up other worker threads waiting for WAL to write + */ + private void signalNextAvailable() { + lock.lock(); + + try { + WALRecord rec = head.get(); + + if (envFailed == null) { + assert rec instanceof FakeRecord : "Expected head FakeRecord, actual head " + + (rec != null ? rec.getClass().getSimpleName() : "null"); + + assert written == lastFsyncPos || mode != WALMode.FSYNC : + "fsync [written=" + written + ", lastFsync=" + lastFsyncPos + ']'; + } + + fileIO = null; + + nextSegment.signalAll(); + } + finally { + lock.unlock(); + } + } + + /** + * @throws IgniteCheckedException If failed. + */ + private void awaitNext() throws IgniteCheckedException { + lock.lock(); + + try { + while (fileIO != null) + U.awaitQuiet(nextSegment); + } + finally { + lock.unlock(); + } + } + + /** + * @param pos Position in file to start write from. May be checked against actual position to wait previous + * writes to complete + * @param buf Buffer to write to file + * @throws StorageException If failed. + * @throws IgniteCheckedException If failed. + */ + @SuppressWarnings("TooBroadScope") + private void writeBuffer(long pos, ByteBuffer buf) throws StorageException, IgniteCheckedException { + boolean interrupted = false; + + lock.lock(); + + try { + assert fileIO != null : "Writing to a closed segment."; + + checkNode(); + + long lastLogged = U.currentTimeMillis(); + + long logBackoff = 2_000; + + // If we were too fast, need to wait previous writes to complete. + while (written != pos) { + assert written < pos : "written = " + written + ", pos = " + pos; // No one can write further than we are now. + + // Permutation occurred between blocks write operations. + // Order of acquiring lock is not the same as order of write. + long now = U.currentTimeMillis(); + + if (now - lastLogged >= logBackoff) { + if (logBackoff < 60 * 60_000) + logBackoff *= 2; + + U.warn(log, "Still waiting for a concurrent write to complete [written=" + written + + ", pos=" + pos + ", lastFsyncPos=" + lastFsyncPos + ", stop=" + stop.get() + + ", actualPos=" + safePosition() + ']'); + + lastLogged = now; + } + + try { + writeComplete.await(2, TimeUnit.SECONDS); + } + catch (InterruptedException ignore) { + interrupted = true; + } + + checkNode(); + } + + // Do the write. + int size = buf.remaining(); + + assert size > 0 : size; + + try { + assert written == fileIO.position(); + + do { + fileIO.write(buf); + } + while (buf.hasRemaining()); + + written += size; + + metrics.onWalBytesWritten(size); + + assert written == fileIO.position(); + } + catch (IOException e) { + StorageException se = new StorageException("Unable to write", e); + + NodeInvalidator.INSTANCE.invalidate(cctx.kernalContext(), se); + + throw se; + } + } + finally { + writeComplete.signalAll(); + + lock.unlock(); + + if (interrupted) + Thread.currentThread().interrupt(); + } + } + + /** + * @return Safely reads current position of the file channel as String. Will return "null" if channel is null. + */ + private String safePosition() { + FileIO io = this.fileIO; + + if (io == null) + return "null"; + + try { + return String.valueOf(io.position()); + } + catch (IOException e) { + return "{Failed to read channel position: " + e.getMessage() + "}"; + } + } + } + + /** + * Gets WAL record offset relative to the WAL segment file beginning. + * + * @param rec WAL record. + * @return File offset. + */ + private static int recordOffset(WALRecord rec) { + FileWALPointer ptr = (FileWALPointer)rec.position(); + + assert ptr != null; + + return ptr.fileOffset(); + } + + /** + * Fake record is zero-sized record, which is not stored into file. + * Fake record is used for storing position in file {@link WALRecord#position()}. + * Fake record is allowed to have no previous record. + */ + private static final class FakeRecord extends WALRecord { + /** */ + private final boolean stop; + + /** + * @param pos Position. + */ + FakeRecord(FileWALPointer pos, boolean stop) { + position(pos); + + this.stop = stop; + } + + /** {@inheritDoc} */ + @Override public RecordType type() { + return null; + } + + /** {@inheritDoc} */ + @Override public FileWALPointer position() { + return (FileWALPointer) super.position(); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(FakeRecord.class, this, "super", super.toString()); + } + } + + /** + * Iterator over WAL-log. + */ + private class RecordsIterator extends AbstractWalRecordsIterator { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private final File walWorkDir; + + /** */ + private final File walArchiveDir; + + /** */ + private final FileArchiver archiver; + + /** */ + private final FileDecompressor decompressor; + + /** */ + private final DataStorageConfiguration psCfg; + + /** Optional start pointer. */ + @Nullable + private FileWALPointer start; + + /** Optional end pointer. */ + @Nullable + private FileWALPointer end; + + /** + * @param cctx Shared context. + * @param walWorkDir WAL work dir. + * @param walArchiveDir WAL archive dir. + * @param start Optional start pointer. + * @param end Optional end pointer. + * @param psCfg Database configuration. + * @param serializerFactory Serializer factory. + * @param archiver Archiver. + * @param decompressor Decompressor. + *@param log Logger @throws IgniteCheckedException If failed to initialize WAL segment. + */ + private RecordsIterator( + GridCacheSharedContext cctx, + File walWorkDir, + File walArchiveDir, + @Nullable FileWALPointer start, + @Nullable FileWALPointer end, + DataStorageConfiguration psCfg, + @NotNull RecordSerializerFactory serializerFactory, + FileIOFactory ioFactory, + FileArchiver archiver, + FileDecompressor decompressor, + IgniteLogger log + ) throws IgniteCheckedException { + super(log, + cctx, + serializerFactory, + ioFactory, + psCfg.getWalRecordIteratorBufferSize()); + this.walWorkDir = walWorkDir; + this.walArchiveDir = walArchiveDir; + this.psCfg = psCfg; + this.archiver = archiver; + this.start = start; + this.end = end; + this.decompressor = decompressor; + + init(); + + advance(); + } + + /** {@inheritDoc} */ + @Override protected ReadFileHandle initReadHandle( + @NotNull AbstractFileDescriptor desc, + @Nullable FileWALPointer start + ) throws IgniteCheckedException, FileNotFoundException { + if (decompressor != null && !desc.file().exists()) { + FileDescriptor zipFile = new FileDescriptor( + new File(walArchiveDir, FileDescriptor.fileName(desc.idx()) + ".zip")); + + if (!zipFile.file.exists()) { + throw new FileNotFoundException("Both compressed and raw segment files are missing in archive " + + "[segmentIdx=" + desc.idx() + "]"); + } + + decompressor.decompressFile(desc.idx()).get(); + } + + return (ReadFileHandle) super.initReadHandle(desc, start); + } + + /** {@inheritDoc} */ + @Override protected void onClose() throws IgniteCheckedException { + super.onClose(); + + curRec = null; + + final AbstractReadFileHandle handle = closeCurrentWalSegment(); + + if (handle != null && handle.workDir()) + releaseWorkSegment(curWalSegmIdx); + + curWalSegmIdx = Integer.MAX_VALUE; + } + + /** + * @throws IgniteCheckedException If failed to initialize first file handle. + */ + private void init() throws IgniteCheckedException { + AbstractFileDescriptor[] descs = loadFileDescriptors(walArchiveDir); + + if (start != null) { + if (!F.isEmpty(descs)) { + if (descs[0].idx() > start.index()) + throw new IgniteCheckedException("WAL history is too short " + + "[descs=" + Arrays.asList(descs) + ", start=" + start + ']'); + + for (AbstractFileDescriptor desc : descs) { + if (desc.idx() == start.index()) { + curWalSegmIdx = start.index(); + + break; + } + } + + if (curWalSegmIdx == -1) { + long lastArchived = descs[descs.length - 1].idx(); + + if (lastArchived > start.index()) + throw new IgniteCheckedException("WAL history is corrupted (segment is missing): " + start); + + // This pointer may be in work files because archiver did not + // copy the file yet, check that it is not too far forward. + curWalSegmIdx = start.index(); + } + } + else { + // This means that whole checkpoint history fits in one segment in WAL work directory. + // Will start from this index right away. + curWalSegmIdx = start.index(); + } + } + else + curWalSegmIdx = !F.isEmpty(descs) ? descs[0].idx() : 0; + + curWalSegmIdx--; + + if (log.isDebugEnabled()) + log.debug("Initialized WAL cursor [start=" + start + ", end=" + end + ", curWalSegmIdx=" + curWalSegmIdx + ']'); + } + + /** {@inheritDoc} */ + @Override protected AbstractReadFileHandle advanceSegment( + @Nullable final AbstractReadFileHandle curWalSegment + ) throws IgniteCheckedException { + if (curWalSegment != null) { + curWalSegment.close(); + + if (curWalSegment.workDir()) + releaseWorkSegment(curWalSegment.idx()); + + } + + // We are past the end marker. + if (end != null && curWalSegmIdx + 1 > end.index()) + return null; //stop iteration + + curWalSegmIdx++; + + FileDescriptor fd; + + boolean readArchive = canReadArchiveOrReserveWork(curWalSegmIdx); + + if (readArchive) + fd = new FileDescriptor(new File(walArchiveDir, FileDescriptor.fileName(curWalSegmIdx))); + else { + long workIdx = curWalSegmIdx % psCfg.getWalSegments(); + + fd = new FileDescriptor( + new File(walWorkDir, FileDescriptor.fileName(workIdx)), + curWalSegmIdx); + } + + if (log.isDebugEnabled()) + log.debug("Reading next file [absIdx=" + curWalSegmIdx + ", file=" + fd.file().getAbsolutePath() + ']'); + + ReadFileHandle nextHandle; + + try { + nextHandle = initReadHandle(fd, start != null && curWalSegmIdx == start.index() ? start : null); + } + catch (FileNotFoundException e) { + if (readArchive) + throw new IgniteCheckedException("Missing WAL segment in the archive", e); + else + nextHandle = null; + } + + if (nextHandle == null) { + if (!readArchive) + releaseWorkSegment(curWalSegmIdx); + } + else + nextHandle.workDir = !readArchive; + + curRec = null; + + return nextHandle; + } + + /** + * @param absIdx Absolute index to check. + * @return
  • {@code True} if we can safely read the archive,
  • {@code false} if the segment has + * not been archived yet. In this case the corresponding work segment is reserved (will not be deleted until + * release). Use {@link #releaseWorkSegment} for unlock
+ */ + private boolean canReadArchiveOrReserveWork(long absIdx) { + return archiver != null && archiver.checkCanReadArchiveOrReserveWorkSegment(absIdx); + } + + /** + * @param absIdx Absolute index to release. + */ + private void releaseWorkSegment(long absIdx) { + if (archiver != null) + archiver.releaseWorkSegment(absIdx); + } + + /** {@inheritDoc} */ + @Override protected AbstractReadFileHandle createReadFileHandle(FileIO fileIO, long idx, + RecordSerializer ser, FileInput in) { + return new ReadFileHandle(fileIO, idx, ser, in); + } + } + + /** + * Flushes current file handle for {@link WALMode#BACKGROUND} WALMode. + * Called periodically from scheduler. + */ + private void doFlush() { + final FileWriteHandle hnd = currentHandle(); + try { + hnd.flush(hnd.head.get(), false); + } + catch (Exception e) { + U.warn(log, "Failed to flush WAL record queue", e); + } + } + + /** + * Scans provided folder for a WAL segment files + * @param walFilesDir directory to scan + * @return found WAL file descriptors + */ + private static FileDescriptor[] loadFileDescriptors(@NotNull final File walFilesDir) throws IgniteCheckedException { + final File[] files = walFilesDir.listFiles(WAL_SEGMENT_COMPACTED_OR_RAW_FILE_FILTER); + + if (files == null) { + throw new IgniteCheckedException("WAL files directory does not not denote a " + + "directory, or if an I/O error occurs: [" + walFilesDir.getAbsolutePath() + "]"); + } + return scan(files); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/SingleSegmentLogicalRecordsIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/SingleSegmentLogicalRecordsIterator.java index 4a846b456fc50..d5c10cf72bc4d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/SingleSegmentLogicalRecordsIterator.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/SingleSegmentLogicalRecordsIterator.java @@ -25,8 +25,10 @@ import org.apache.ignite.internal.pagemem.wal.record.MarshalledRecord; import org.apache.ignite.internal.pagemem.wal.record.WALRecord; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.persistence.file.FileIO; import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory; import org.apache.ignite.internal.processors.cache.persistence.wal.record.RecordTypes; +import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordSerializer; import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordSerializerFactory; import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordSerializerFactoryImpl; import org.apache.ignite.internal.util.typedef.CIX1; @@ -93,8 +95,8 @@ private static RecordSerializerFactory initLogicalRecordsSerializerFactory(GridC } /** {@inheritDoc} */ - @Override protected FileWriteAheadLogManager.ReadFileHandle advanceSegment( - @Nullable FileWriteAheadLogManager.ReadFileHandle curWalSegment) throws IgniteCheckedException { + @Override protected AbstractReadFileHandle advanceSegment( + @Nullable AbstractReadFileHandle curWalSegment) throws IgniteCheckedException { if (segmentInitialized) { closeCurrentWalSegment(); // No advance as we iterate over single segment. @@ -123,6 +125,12 @@ private static RecordSerializerFactory initLogicalRecordsSerializerFactory(GridC advanceC.apply(curRec.get2()); } + /** {@inheritDoc} */ + @Override protected AbstractReadFileHandle createReadFileHandle(FileIO fileIO, long idx, + RecordSerializer ser, FileInput in) { + return new FileWriteAheadLogManager.ReadFileHandle(fileIO, idx, ser, in); + } + /** * */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java index 7740f28fed8ce..712517bbbf90c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java @@ -45,6 +45,7 @@ import org.apache.ignite.internal.processors.cache.persistence.wal.FileInput; import org.apache.ignite.internal.processors.cache.persistence.wal.FileWALPointer; import org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager; +import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordSerializer; import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordSerializerFactoryImpl; import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV1Serializer; import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor; @@ -153,7 +154,7 @@ private void init( final boolean workDir, @Nullable final File[] walFiles) throws IgniteCheckedException { if (walFilesDir != null) { - FileWriteAheadLogManager.FileDescriptor[] descs = loadFileDescriptors(walFilesDir); + FileWriteAheadLogManager.FileDescriptor[] descs = FileWriteAheadLogManager.loadFileDescriptors(walFilesDir); curWalSegmIdx = !F.isEmpty(descs) ? descs[0].getIdx() : 0; this.walFilesDir = walFilesDir; } @@ -224,8 +225,8 @@ private List scanIndexesFromFileHeaders } /** {@inheritDoc} */ - @Override protected FileWriteAheadLogManager.ReadFileHandle advanceSegment( - @Nullable final FileWriteAheadLogManager.ReadFileHandle curWalSegment) throws IgniteCheckedException { + @Override protected AbstractReadFileHandle advanceSegment( + @Nullable final AbstractReadFileHandle curWalSegment) throws IgniteCheckedException { if (curWalSegment != null) curWalSegment.close(); @@ -380,4 +381,10 @@ private DataEntry postProcessDataEntry( curWalSegmIdx = Integer.MAX_VALUE; } + + /** {@inheritDoc} */ + @Override protected AbstractReadFileHandle createReadFileHandle(FileIO fileIO, long idx, + RecordSerializer ser, FileInput in) { + return new FileWriteAheadLogManager.ReadFileHandle(fileIO, idx, ser, in); + } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushDefaultSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushDefaultSelfTest.java index d60241e0f3894..94e7e258b298b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushDefaultSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushDefaultSelfTest.java @@ -30,6 +30,6 @@ public class IgniteWalFlushDefaultSelfTest extends IgniteWalFlushMultiNodeFailov /** {@inheritDoc} */ @Override protected WALMode walMode() { - return WALMode.DEFAULT; + return WALMode.FSYNC; } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java index 3bd1582b02c42..9dda03d88c736 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java @@ -731,7 +731,7 @@ else if (val instanceof BinaryObject) { * @throws Exception if failed. */ public void testFillWalForExactSegmentsCount() throws Exception { - customWalMode = WALMode.DEFAULT; + customWalMode = WALMode.FSYNC; final CountDownLatch reqSegments = new CountDownLatch(15); final Ignite ignite = startGrid("node0"); @@ -780,7 +780,7 @@ public void testFillWalForExactSegmentsCount() throws Exception { * @throws Exception if failed. */ public void testReadEmptyWal() throws Exception { - customWalMode = WALMode.DEFAULT; + customWalMode = WALMode.FSYNC; final Ignite ignite = startGrid("node0"); From 805ac96c9715f4e58b68dfffe6b4fe260644e1de Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Tue, 20 Feb 2018 10:27:40 +0700 Subject: [PATCH 027/314] IGNITE-7725 REST: Enhanced GET_OR_CREATE_CACHE command with optional "templateName", "backups", "cacheGroup", "dataRegion" and "writeSynchronizationMode" options. (cherry picked from commit ad0cd50) --- .../JettyRestProcessorAbstractSelfTest.java | 611 +++++++++++------- .../apache/ignite/internal/IgniteKernal.java | 16 +- .../cache/CacheConfigurationOverride.java | 165 +++++ .../processors/cache/GridCacheProcessor.java | 29 +- .../processors/cache/GridCacheUtils.java | 2 +- .../cache/GridCacheCommandHandler.java | 46 +- .../rest/request/GridRestCacheRequest.java | 40 +- .../http/jetty/GridJettyRestHandler.java | 101 ++- 8 files changed, 752 insertions(+), 258 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheConfigurationOverride.java diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java index 13613efde6e85..a33f0199eaa3f 100644 --- a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java @@ -31,19 +31,25 @@ import java.util.Collection; import java.util.HashMap; import java.util.Iterator; +import java.util.LinkedHashMap; import java.util.Locale; import java.util.Map; import java.util.Set; import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; + import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; + import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.CacheWriteSynchronizationMode; import org.apache.ignite.cache.query.SqlQuery; 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.DataRegionConfiguration; +import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.configuration.FileSystemConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.igfs.IgfsIpcEndpointConfiguration; @@ -139,7 +145,13 @@ import org.apache.ignite.testframework.GridTestUtils; import static org.apache.ignite.IgniteSystemProperties.IGNITE_JETTY_PORT; +import static org.apache.ignite.cache.CacheMode.PARTITIONED; +import static org.apache.ignite.cache.CacheMode.REPLICATED; +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_ASYNC; +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; import static org.apache.ignite.internal.IgniteVersionUtils.VER_STR; +import static org.apache.ignite.internal.processors.query.QueryUtils.TEMPLATE_PARTITIONED; +import static org.apache.ignite.internal.processors.query.QueryUtils.TEMPLATE_REPLICATED; /** * Tests for Jetty REST protocol. @@ -228,6 +240,31 @@ protected String content(Map params) throws Exception { return buf.toString(); } + /** + * @param cacheName Optional cache name. + * @param cmd REST command. + * @param params Command parameters. + * @return Returned content. + * @throws Exception If failed. + */ + protected String content(String cacheName, GridRestCommand cmd, String... params) throws Exception { + Map paramsMap = new LinkedHashMap<>(); + + if (cacheName != null) + paramsMap.put("cacheName", cacheName); + + paramsMap.put("cmd", cmd.key()); + + if (params != null) { + assertEquals(0, params.length % 2); + + for (int i = 0; i < params.length; i += 2) + paramsMap.put(params[i], params[i + 1]); + } + + return content(paramsMap); + } + /** * @param content Content to check. */ @@ -359,7 +396,7 @@ protected JsonNode jsonTaskResult(String content) throws IOException { public void testGet() throws Exception { jcache().put("getKey", "getVal"); - String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_GET.key(), "key", "getKey")); + String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_GET, "key", "getKey"); info("Get command result: " + ret); @@ -376,7 +413,7 @@ public void testNullMapKeyAndValue() throws Exception { jcache().put("mapKey1", map1); - String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_GET.key(), "key", "mapKey1")); + String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_GET, "key", "mapKey1"); info("Get command result: " + ret); @@ -390,7 +427,7 @@ public void testNullMapKeyAndValue() throws Exception { jcache().put("mapKey2", map2); - ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_GET.key(), "key", "mapKey2")); + ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_GET, "key", "mapKey2"); info("Get command result: " + ret); @@ -407,7 +444,7 @@ public void testSimpleObject() throws Exception { jcache().put("simplePersonKey", p); - String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_GET.key(), "key", "simplePersonKey")); + String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_GET, "key", "simplePersonKey"); info("Get command result: " + ret); @@ -434,7 +471,7 @@ public void testDate() throws Exception { jcache().put("utilDateKey", utilDate); - String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_GET.key(), "key", "utilDateKey")); + String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_GET, "key", "utilDateKey"); info("Get command result: " + ret); @@ -444,7 +481,7 @@ public void testDate() throws Exception { jcache().put("sqlDateKey", sqlDate); - ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_GET.key(), "key", "sqlDateKey")); + ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_GET, "key", "sqlDateKey"); info("Get SQL result: " + ret); @@ -452,7 +489,7 @@ public void testDate() throws Exception { jcache().put("timestampKey", new java.sql.Timestamp(utilDate.getTime())); - ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_GET.key(), "key", "timestampKey")); + ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_GET, "key", "timestampKey"); info("Get timestamp: " + ret); @@ -467,7 +504,7 @@ public void testUUID() throws Exception { jcache().put("uuidKey", uuid); - String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_GET.key(), "key", "uuidKey")); + String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_GET, "key", "uuidKey"); info("Get command result: " + ret); @@ -477,7 +514,7 @@ public void testUUID() throws Exception { jcache().put("igniteUuidKey", igniteUuid); - ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_GET.key(), "key", "igniteUuidKey")); + ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_GET, "key", "igniteUuidKey"); info("Get command result: " + ret); @@ -492,7 +529,7 @@ public void testTuple() throws Exception { jcache().put("tupleKey", t); - String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_GET.key(), "key", "tupleKey")); + String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_GET, "key", "tupleKey"); info("Get command result: " + ret); @@ -510,7 +547,7 @@ public void testCacheSize() throws Exception { jcache().put("getKey", "getVal"); - String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_SIZE.key())); + String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_SIZE); info("Size command result: " + ret); @@ -521,7 +558,7 @@ public void testCacheSize() throws Exception { * @throws Exception If failed. */ public void testIgniteName() throws Exception { - String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.NAME.key())); + String ret = content(null, GridRestCommand.NAME); info("Name command result: " + ret); @@ -529,20 +566,94 @@ public void testIgniteName() throws Exception { } /** - * @throws Exception If failed. + * @param cacheName Cache name to create. + * @param mode Expected cache mode. + * @param backups Expected number of backups. + * @param wrtSync Expected cache write synchronization mode. + * @param params Optional cache params. */ - public void testGetOrCreateCache() throws Exception { - String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.GET_OR_CREATE_CACHE.key(), "cacheName", "testCache")); + private void checkGetOrCreateAndDestroy( + String cacheName, + CacheMode mode, + int backups, + CacheWriteSynchronizationMode wrtSync, + String cacheGroup, + String dataRegion, + String... params + ) throws Exception { + String ret = content(cacheName, GridRestCommand.GET_OR_CREATE_CACHE, params); - info("Name command result: " + ret); + info("GetOrCreateCache command result: " + ret); + + IgniteCache cache = grid(0).cache(cacheName); + + cache.put("1", "1"); + + CacheConfiguration ccfg = cache.getConfiguration(CacheConfiguration.class); + + assertEquals(backups, ccfg.getBackups()); + assertEquals(mode, ccfg.getCacheMode()); + assertEquals(wrtSync, ccfg.getWriteSynchronizationMode()); + + if (!F.isEmpty(cacheGroup)) + assertEquals(cacheGroup, ccfg.getGroupName()); - grid(0).cache("testCache").put("1", "1"); + if (!F.isEmpty(dataRegion)) + assertEquals(dataRegion, ccfg.getDataRegionName()); - ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.DESTROY_CACHE.key(), "cacheName", "testCache")); + ret = content(cacheName, GridRestCommand.DESTROY_CACHE); assertTrue(jsonResponse(ret).isNull()); + assertNull(grid(0).cache(cacheName)); + } - assertNull(grid(0).cache("testCache")); + /** + * @throws Exception If failed. + */ + public void testGetOrCreateCache() throws Exception { + checkGetOrCreateAndDestroy("testCache", PARTITIONED, 0, FULL_SYNC, null, null); + + checkGetOrCreateAndDestroy("testCache", PARTITIONED, 3, FULL_SYNC, null, null, + "backups", "3" + ); + + checkGetOrCreateAndDestroy("testCache", PARTITIONED, 2, FULL_ASYNC, null, null, + "backups", "2", + "writeSynchronizationMode", FULL_ASYNC.name() + ); + + checkGetOrCreateAndDestroy("testCache", REPLICATED, Integer.MAX_VALUE, FULL_ASYNC, null, null, + "templateName", TEMPLATE_REPLICATED, + "writeSynchronizationMode", FULL_ASYNC.name() + ); + + checkGetOrCreateAndDestroy("testCache", REPLICATED, Integer.MAX_VALUE, FULL_ASYNC, null, null, + "templateName", TEMPLATE_REPLICATED, + "backups", "0", + "writeSynchronizationMode", FULL_ASYNC.name() + ); + + checkGetOrCreateAndDestroy("testCache", PARTITIONED, 1, FULL_ASYNC, "testGroup", null, + "templateName", TEMPLATE_PARTITIONED, + "backups", "1", + "writeSynchronizationMode", FULL_ASYNC.name(), + "cacheGroup", "testGroup" + ); + + checkGetOrCreateAndDestroy("testCache", PARTITIONED, 2, FULL_ASYNC, null, "testDataRegion", + "templateName", TEMPLATE_PARTITIONED, + "backups", "2", + "writeSynchronizationMode", FULL_ASYNC.name(), + "dataRegion", "testDataRegion" + ); + + checkGetOrCreateAndDestroy("testCache", PARTITIONED, 3, FULL_ASYNC, "testGroup", "testDataRegion", + "templateName", TEMPLATE_PARTITIONED, + "backups", "3", + "writeSynchronizationMode", FULL_ASYNC.name(), + "cacheGroup", "testGroup", + "dataRegion", "testDataRegion" + ); } /** @@ -553,7 +664,10 @@ public void testGetAll() throws Exception { jcache().putAll(entries); - String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_GET_ALL.key(), "k1", "getKey1", "k2", "getKey2")); + String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_GET_ALL, + "k1", "getKey1", + "k2", "getKey2" + ); info("Get all command result: " + ret); @@ -568,7 +682,7 @@ public void testGetAll() throws Exception { * @throws Exception If failed. */ public void testIncorrectPut() throws Exception { - String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_PUT.key(), "key", "key0")); + String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_PUT, "key", "key0"); assertResponseContainsError(ret, "Failed to handle request: [req=CACHE_PUT, err=Failed to find mandatory parameter in request: val]"); @@ -580,7 +694,7 @@ public void testIncorrectPut() throws Exception { public void testContainsKey() throws Exception { grid(0).cache(DEFAULT_CACHE_NAME).put("key0", "val0"); - String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_CONTAINS_KEY.key(), "key", "key0")); + String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_CONTAINS_KEY, "key", "key0"); assertCacheOperation(ret, true); } @@ -588,12 +702,14 @@ public void testContainsKey() throws Exception { /** * @throws Exception If failed. */ - public void testContainesKeys() throws Exception { + public void testContainsKeys() throws Exception { grid(0).cache(DEFAULT_CACHE_NAME).put("key0", "val0"); grid(0).cache(DEFAULT_CACHE_NAME).put("key1", "val1"); - String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_CONTAINS_KEYS.key(), - "k1", "key0", "k2", "key1")); + String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_CONTAINS_KEYS, + "k1", "key0", + "k2", "key1" + ); assertCacheBulkOperation(ret, true); } @@ -604,7 +720,10 @@ public void testContainesKeys() throws Exception { public void testGetAndPut() throws Exception { grid(0).cache(DEFAULT_CACHE_NAME).put("key0", "val0"); - String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_GET_AND_PUT.key(), "key", "key0", "val", "val1")); + String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_GET_AND_PUT, + "key", "key0", + "val", "val1" + ); assertCacheOperation(ret, "val0"); @@ -617,8 +736,10 @@ public void testGetAndPut() throws Exception { public void testGetAndPutIfAbsent() throws Exception { grid(0).cache(DEFAULT_CACHE_NAME).put("key0", "val0"); - String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_GET_AND_PUT_IF_ABSENT.key(), - "key", "key0", "val", "val1")); + String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_GET_AND_PUT_IF_ABSENT, + "key", "key0", + "val", "val1" + ); assertCacheOperation(ret, "val0"); @@ -629,8 +750,10 @@ public void testGetAndPutIfAbsent() throws Exception { * @throws Exception If failed. */ public void testPutIfAbsent2() throws Exception { - String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_PUT_IF_ABSENT.key(), - "key", "key0", "val", "val1")); + String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_PUT_IF_ABSENT, + "key", "key0", + "val", "val1" + ); assertCacheOperation(ret, true); @@ -643,15 +766,19 @@ public void testPutIfAbsent2() throws Exception { public void testRemoveValue() throws Exception { grid(0).cache(DEFAULT_CACHE_NAME).put("key0", "val0"); - String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_REMOVE_VALUE.key(), - "key", "key0", "val", "val1")); + String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_REMOVE_VALUE, + "key", "key0", + "val", "val1" + ); assertCacheOperation(ret, false); assertEquals("val0", grid(0).cache(DEFAULT_CACHE_NAME).get("key0")); - ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_REMOVE_VALUE.key(), - "key", "key0", "val", "val0")); + ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_REMOVE_VALUE, + "key", "key0", + "val", "val0" + ); assertCacheOperation(ret, true); @@ -664,8 +791,7 @@ public void testRemoveValue() throws Exception { public void testGetAndRemove() throws Exception { grid(0).cache(DEFAULT_CACHE_NAME).put("key0", "val0"); - String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_GET_AND_REMOVE.key(), - "key", "key0")); + String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_GET_AND_REMOVE, "key", "key0"); assertCacheOperation(ret, "val0"); @@ -678,15 +804,21 @@ public void testGetAndRemove() throws Exception { public void testReplaceValue() throws Exception { grid(0).cache(DEFAULT_CACHE_NAME).put("key0", "val0"); - String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_REPLACE_VALUE.key(), - "key", "key0", "val", "val1", "val2", "val2")); + String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_REPLACE_VALUE, + "key", "key0", + "val", "val1", + "val2", "val2" + ); assertCacheOperation(ret, false); assertEquals("val0", grid(0).cache(DEFAULT_CACHE_NAME).get("key0")); - ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_REPLACE_VALUE.key(), - "key", "key0", "val", "val1", "val2", "val0")); + ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_REPLACE_VALUE, + "key", "key0", + "val", "val1", + "val2", "val0" + ); assertCacheOperation(ret, true); @@ -699,8 +831,10 @@ public void testReplaceValue() throws Exception { public void testGetAndReplace() throws Exception { grid(0).cache(DEFAULT_CACHE_NAME).put("key0", "val0"); - String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_GET_AND_REPLACE.key(), - "key", "key0", "val", "val1")); + String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_GET_AND_REPLACE, + "key", "key0", + "val", "val1" + ); assertCacheOperation(ret, "val0"); @@ -723,8 +857,10 @@ public void testDeactivateActivate() throws Exception { * @throws Exception If failed. */ public void testPut() throws Exception { - String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_PUT.key(), - "key", "putKey", "val", "putVal")); + String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_PUT, + "key", "putKey", + "val", "putVal" + ); info("Put command result: " + ret); @@ -737,8 +873,11 @@ public void testPut() throws Exception { * @throws Exception If failed. */ public void testPutWithExpiration() throws Exception { - String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_PUT.key(), - "key", "putKey", "val", "putVal", "exp", "2000")); + String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_PUT, + "key", "putKey", + "val", "putVal", + "exp", "2000" + ); assertCacheOperation(ret, true); @@ -755,8 +894,10 @@ public void testPutWithExpiration() throws Exception { public void testAdd() throws Exception { jcache().put("addKey1", "addVal1"); - String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_ADD.key(), - "key", "addKey2", "val", "addVal2")); + String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_ADD, + "key", "addKey2", + "val", "addVal2" + ); assertCacheOperation(ret, true); @@ -768,8 +909,11 @@ public void testAdd() throws Exception { * @throws Exception If failed. */ public void testAddWithExpiration() throws Exception { - String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_ADD.key(), - "key", "addKey", "val", "addVal", "exp", "2000")); + String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_ADD, + "key", "addKey", + "val", "addVal", + "exp", "2000" + ); assertCacheOperation(ret, true); @@ -784,14 +928,12 @@ public void testAddWithExpiration() throws Exception { * @throws Exception If failed. */ public void testPutAll() throws Exception { - Map map = F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_PUT_ALL.key()); - - map.put("k1", "putKey1"); - map.put("k2", "putKey2"); - map.put("v1", "putVal1"); - map.put("v2", "putVal2"); - - String ret = content(map); + String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_PUT_ALL, + "k1", "putKey1", + "k2", "putKey2", + "v1", "putVal1", + "v2", "putVal2" + ); info("Put all command result: " + ret); @@ -809,8 +951,7 @@ public void testRemove() throws Exception { assertEquals("rmvVal", jcache().localPeek("rmvKey")); - String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_REMOVE.key(), - "key", "rmvKey")); + String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_REMOVE, "key", "rmvKey"); info("Remove command result: " + ret); @@ -833,8 +974,10 @@ public void testRemoveAll() throws Exception { assertEquals("rmvVal3", jcache().localPeek("rmvKey3")); assertEquals("rmvVal4", jcache().localPeek("rmvKey4")); - String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_REMOVE_ALL.key(), - "k1", "rmvKey1", "k2", "rmvKey2")); + String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_REMOVE_ALL, + "k1", "rmvKey1", + "k2", "rmvKey2" + ); info("Remove all command result: " + ret); @@ -845,7 +988,7 @@ public void testRemoveAll() throws Exception { assertCacheBulkOperation(ret, true); - ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_REMOVE_ALL.key())); + ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_REMOVE_ALL); info("Remove all command result: " + ret); @@ -866,8 +1009,11 @@ public void testCas() throws Exception { assertEquals("casOldVal", jcache().localPeek("casKey")); - String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_CAS.key(), - "key", "casKey", "val2", "casOldVal", "val1", "casNewVal")); + String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_CAS, + "key", "casKey", + "val2", "casOldVal", + "val1", "casNewVal" + ); info("CAS command result: " + ret); @@ -886,8 +1032,10 @@ public void testReplace() throws Exception { assertEquals("repOldVal", jcache().localPeek("repKey")); - String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_REPLACE.key(), - "key", "repKey", "val", "repVal")); + String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_REPLACE, + "key", "repKey", + "val", "repVal" + ); info("Replace command result: " + ret); @@ -904,8 +1052,11 @@ public void testReplaceWithExpiration() throws Exception { assertEquals("replaceVal", jcache().get("replaceKey")); - String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_REPLACE.key(), - "key", "replaceKey", "val", "replaceValNew", "exp", "2000")); + String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_REPLACE, + "key", "replaceKey", + "val", "replaceValNew", + "exp", "2000" + ); assertCacheOperation(ret, true); @@ -923,8 +1074,10 @@ public void testReplaceWithExpiration() throws Exception { public void testAppend() throws Exception { jcache().put("appendKey", "appendVal"); - String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_APPEND.key(), - "key", "appendKey", "val", "_suffix")); + String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_APPEND, + "key", "appendKey", + "val", "_suffix" + ); assertCacheOperation(ret, true); @@ -937,8 +1090,10 @@ public void testAppend() throws Exception { public void testPrepend() throws Exception { jcache().put("prependKey", "prependVal"); - String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_PREPEND.key(), - "key", "prependKey", "val", "prefix_")); + String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_PREPEND, + "key", "prependKey", + "val", "prefix_" + ); assertCacheOperation(ret, true); @@ -949,15 +1104,21 @@ public void testPrepend() throws Exception { * @throws Exception If failed. */ public void testIncrement() throws Exception { - String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.ATOMIC_INCREMENT.key(), - "key", "incrKey", "init", "2", "delta", "3")); + String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.ATOMIC_INCREMENT, + "key", "incrKey", + "init", "2", + "delta", "3" + ); JsonNode res = jsonResponse(ret); assertEquals(5, res.asInt()); assertEquals(5, grid(0).atomicLong("incrKey", 0, true).get()); - ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.ATOMIC_INCREMENT.key(), "key", "incrKey", "delta", "10")); + ret = content(DEFAULT_CACHE_NAME, GridRestCommand.ATOMIC_INCREMENT, + "key", "incrKey", + "delta", "10" + ); res = jsonResponse(ret); @@ -969,16 +1130,21 @@ public void testIncrement() throws Exception { * @throws Exception If failed. */ public void testDecrement() throws Exception { - String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.ATOMIC_DECREMENT.key(), - "key", "decrKey", "init", "15", "delta", "10")); + String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.ATOMIC_DECREMENT, + "key", "decrKey", + "init", "15", + "delta", "10" + ); JsonNode res = jsonResponse(ret); assertEquals(5, res.asInt()); assertEquals(5, grid(0).atomicLong("decrKey", 0, true).get()); - ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.ATOMIC_DECREMENT.key(), - "key", "decrKey", "delta", "3")); + ret = content(DEFAULT_CACHE_NAME, GridRestCommand.ATOMIC_DECREMENT, + "key", "decrKey", + "delta", "3" + ); res = jsonResponse(ret); @@ -994,8 +1160,10 @@ public void testCar() throws Exception { assertEquals("casOldVal", jcache().localPeek("casKey")); - String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_CAS.key(), - "key", "casKey", "val2", "casOldVal")); + String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_CAS, + "key", "casKey", + "val2", "casOldVal" + ); info("CAR command result: " + ret); @@ -1010,8 +1178,10 @@ public void testCar() throws Exception { public void testPutIfAbsent() throws Exception { assertNull(jcache().localPeek("casKey")); - String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_CAS.key(), - "key", "casKey", "val1", "casNewVal")); + String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_CAS, + "key", "casKey", + "val1", "casNewVal" + ); info("PutIfAbsent command result: " + ret); @@ -1028,7 +1198,7 @@ public void testCasRemove() throws Exception { assertEquals("casVal", jcache().localPeek("casKey")); - String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_CAS.key(), "key", "casKey")); + String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_CAS, "key", "casKey"); info("CAS Remove command result: " + ret); @@ -1041,7 +1211,7 @@ public void testCasRemove() throws Exception { * @throws Exception If failed. */ public void testMetrics() throws Exception { - String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_METRICS.key())); + String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_METRICS); info("Cache metrics command result: " + ret); @@ -1136,32 +1306,34 @@ public void testMetadataLocal() throws Exception { Collection metas = cache.context().queries().sqlMetadata(); - assertEquals(5, metas.size()); + // TODO: IGNITE-7740 uncomment after IGNITE-7740 will be fixed. + // int cachesCnt = grid(0).cacheNames().size(); + // assertEquals(cachesCnt, metas.size()); - String ret = content(F.asMap("cacheName", "", "cmd", GridRestCommand.CACHE_METADATA.key())); + String ret = content("", GridRestCommand.CACHE_METADATA); info("Cache metadata: " + ret); - JsonNode arrResponse = jsonResponse(ret); + JsonNode arrRes = jsonResponse(ret); - assertEquals(5, arrResponse.size()); + // TODO: IGNITE-7740 uncomment after IGNITE-7740 will be fixed. + // assertEquals(cachesCnt, arrRes.size()); - testMetadata(metas, arrResponse); + testMetadata(metas, arrRes); Collection dfltCacheMeta = cache.context().queries().sqlMetadata(); - ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_METADATA.key())); + ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_METADATA); info("Cache metadata: " + ret); - arrResponse = jsonResponse(ret); + arrRes = jsonResponse(ret); - assertEquals(1, arrResponse.size()); + assertEquals(1, arrRes.size()); - testMetadata(dfltCacheMeta, arrResponse); + testMetadata(dfltCacheMeta, arrRes); - assertResponseContainsError(content( - F.asMap("cacheName", "nonExistingCacheName", "cmd", GridRestCommand.CACHE_METADATA.key()))); + assertResponseContainsError(content("nonExistingCacheName", GridRestCommand.CACHE_METADATA)); } /** @@ -1177,36 +1349,39 @@ public void testMetadataRemote() throws Exception { Collection metas = c.context().queries().sqlMetadata(); - String ret = content(F.asMap("cacheName", "", "cmd", GridRestCommand.CACHE_METADATA.key())); + String ret = content("", GridRestCommand.CACHE_METADATA); info("Cache metadata: " + ret); - JsonNode arrResponse = jsonResponse(ret); + JsonNode arrRes = jsonResponse(ret); - assertEquals(6, arrResponse.size()); + // TODO: IGNITE-7740 uncomment after IGNITE-7740 will be fixed. + // int cachesCnt = grid(1).cacheNames().size(); + // assertEquals(cachesCnt, arrRes.size()); - testMetadata(metas, arrResponse); + testMetadata(metas, arrRes); - ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, - "cmd", GridRestCommand.CACHE_METADATA.key(), "cacheName", "person")); + ret = content("person", GridRestCommand.CACHE_METADATA); info("Cache metadata with cacheName parameter: " + ret); - arrResponse = jsonResponse(ret); + arrRes = jsonResponse(ret); - assertEquals(1, arrResponse.size()); + assertEquals(1, arrRes.size()); - testMetadata(metas, arrResponse); + testMetadata(metas, arrRes); - assertResponseContainsError(content( - F.asMap("cacheName", "nonExistingCacheName", "cmd", GridRestCommand.CACHE_METADATA.key()))); + assertResponseContainsError(content("nonExistingCacheName", GridRestCommand.CACHE_METADATA)); } /** * @throws Exception If failed. */ public void testTopology() throws Exception { - String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.TOPOLOGY.key(), "attr", "false", "mtr", "false")); + String ret = content(null, GridRestCommand.TOPOLOGY, + "attr", "false", + "mtr", "false" + ); info("Topology command result: " + ret); @@ -1229,7 +1404,7 @@ public void testTopology() throws Exception { for (JsonNode cache : caches) { String cacheName0 = cache.get("name").asText(); - final String cacheName = cacheName0.equals("") ? null : cacheName0; + final String cacheName = cacheName0.isEmpty() ? null : cacheName0; IgniteCacheProxy publicCache = F.find(publicCaches, null, new P1>() { @Override public boolean apply(IgniteCacheProxy c) { @@ -1250,8 +1425,11 @@ public void testTopology() throws Exception { * @throws Exception If failed. */ public void testNode() throws Exception { - String ret = content(F.asMap("cmd", GridRestCommand.NODE.key(), "attr", "true", "mtr", "true", "id", - grid(0).localNode().id().toString())); + String ret = content(null, GridRestCommand.NODE, + "attr", "true", + "mtr", "true", + "id", grid(0).localNode().id().toString() + ); info("Topology command result: " + ret); @@ -1260,7 +1438,11 @@ public void testNode() throws Exception { assertTrue(res.get("attributes").isObject()); assertTrue(res.get("metrics").isObject()); - ret = content(F.asMap("cmd", GridRestCommand.NODE.key(), "attr", "false", "mtr", "false", "ip", LOC_HOST)); + ret = content(null, GridRestCommand.NODE, + "attr", "false", + "mtr", "false", + "ip", LOC_HOST + ); info("Topology command result: " + ret); @@ -1269,8 +1451,12 @@ public void testNode() throws Exception { assertTrue(res.get("attributes").isNull()); assertTrue(res.get("metrics").isNull()); - ret = content(F.asMap("cmd", GridRestCommand.NODE.key(), "attr", "false", "mtr", "false", "ip", LOC_HOST, "id", - UUID.randomUUID().toString())); + ret = content(null, GridRestCommand.NODE, + "attr", "false", + "mtr", "false", + "ip", LOC_HOST, + "id", UUID.randomUUID().toString() + ); info("Topology command result: " + ret); @@ -1287,14 +1473,14 @@ public void testNode() throws Exception { * @throws Exception If failed. */ public void testExe() throws Exception { - String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.EXE.key())); + String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.EXE); info("Exe command result: " + ret); assertResponseContainsError(ret); // Attempt to execute unknown task (UNKNOWN_TASK) will result in exception on server. - ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.EXE.key(), "name", "UNKNOWN_TASK")); + ret = content(DEFAULT_CACHE_NAME, GridRestCommand.EXE, "name", "UNKNOWN_TASK"); info("Exe command result: " + ret); @@ -1303,7 +1489,7 @@ public void testExe() throws Exception { grid(0).compute().localDeployTask(TestTask1.class, TestTask1.class.getClassLoader()); grid(0).compute().localDeployTask(TestTask2.class, TestTask2.class.getClassLoader()); - ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.EXE.key(), "name", TestTask1.class.getName())); + ret = content(DEFAULT_CACHE_NAME, GridRestCommand.EXE, "name", TestTask1.class.getName()); info("Exe command result: " + ret); @@ -1311,7 +1497,7 @@ public void testExe() throws Exception { assertTrue(res.isNull()); - ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.EXE.key(), "name", TestTask2.class.getName())); + ret = content(DEFAULT_CACHE_NAME, GridRestCommand.EXE, "name", TestTask2.class.getName()); info("Exe command result: " + ret); @@ -1319,7 +1505,7 @@ public void testExe() throws Exception { assertEquals(TestTask2.RES, res.asText()); - ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.RESULT.key())); + ret = content(DEFAULT_CACHE_NAME, GridRestCommand.RESULT); info("Exe command result: " + ret); @@ -1689,7 +1875,7 @@ public void testVisorGateway() throws Exception { * @throws Exception If failed. */ public void testVersion() throws Exception { - String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.VERSION.key())); + String ret = content(null, GridRestCommand.VERSION); JsonNode res = jsonResponse(ret); @@ -1702,16 +1888,13 @@ public void testVersion() throws Exception { public void testQueryArgs() throws Exception { String qry = "salary > ? and salary <= ?"; - Map params = new HashMap<>(); - params.put("cmd", GridRestCommand.EXECUTE_SQL_QUERY.key()); - params.put("type", "Person"); - params.put("pageSize", "10"); - params.put("cacheName", "person"); - params.put("qry", URLEncoder.encode(qry, CHARSET)); - params.put("arg1", "1000"); - params.put("arg2", "2000"); - - String ret = content(params); + String ret = content("person", GridRestCommand.EXECUTE_SQL_QUERY, + "type", "Person", + "pageSize", "10", + "qry", URLEncoder.encode(qry, CHARSET), + "arg1", "1000", + "arg2", "2000" + ); JsonNode items = jsonResponse(ret).get("items"); @@ -1724,12 +1907,10 @@ public void testQueryArgs() throws Exception { * @throws Exception If failed. */ public void testQueryScan() throws Exception { - Map params = new HashMap<>(); - params.put("cmd", GridRestCommand.EXECUTE_SCAN_QUERY.key()); - params.put("pageSize", "10"); - params.put("cacheName", "person"); - - String ret = content(params); + String ret = content("person", GridRestCommand.EXECUTE_SCAN_QUERY, + "pageSize", "10", + "cacheName", "person" + ); JsonNode items = jsonResponse(ret).get("items"); @@ -1742,13 +1923,10 @@ public void testQueryScan() throws Exception { * @throws Exception If failed. */ public void testFilterQueryScan() throws Exception { - Map params = new HashMap<>(); - params.put("cmd", GridRestCommand.EXECUTE_SCAN_QUERY.key()); - params.put("pageSize", "10"); - params.put("cacheName", "person"); - params.put("className", ScanFilter.class.getName()); - - String ret = content(params); + String ret = content("person", GridRestCommand.EXECUTE_SCAN_QUERY, + "pageSize", "10", + "className", ScanFilter.class.getName() + ); JsonNode items = jsonResponse(ret).get("items"); @@ -1763,13 +1941,10 @@ public void testFilterQueryScan() throws Exception { public void testIncorrectFilterQueryScan() throws Exception { String clsName = ScanFilter.class.getName() + 1; - Map params = new HashMap<>(); - params.put("cmd", GridRestCommand.EXECUTE_SCAN_QUERY.key()); - params.put("pageSize", "10"); - params.put("cacheName", "person"); - params.put("className", clsName); - - String ret = content(params); + String ret = content("person", GridRestCommand.EXECUTE_SCAN_QUERY, + "pageSize", "10", + "className", clsName + ); assertResponseContainsError(ret, "Failed to find target class: " + clsName); } @@ -1782,21 +1957,20 @@ public void testQuery() throws Exception { grid(0).cache(DEFAULT_CACHE_NAME).put("2", "2"); grid(0).cache(DEFAULT_CACHE_NAME).put("3", "3"); - Map params = new HashMap<>(); - params.put("cacheName", DEFAULT_CACHE_NAME); - params.put("cmd", GridRestCommand.EXECUTE_SQL_QUERY.key()); - params.put("type", "String"); - params.put("pageSize", "1"); - params.put("qry", URLEncoder.encode("select * from String", CHARSET)); - - String ret = content(params); + String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.EXECUTE_SQL_QUERY, + "type", "String", + "pageSize", "1", + "qry", URLEncoder.encode("select * from String", CHARSET) + ); JsonNode qryId = jsonResponse(ret).get("queryId"); assertFalse(jsonResponse(ret).get("queryId").isNull()); - ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.FETCH_SQL_QUERY.key(), - "pageSize", "1", "qryId", qryId.asText())); + ret = content(DEFAULT_CACHE_NAME, GridRestCommand.FETCH_SQL_QUERY, + "pageSize", "1", + "qryId", qryId.asText() + ); JsonNode res = jsonResponse(ret); @@ -1805,8 +1979,10 @@ public void testQuery() throws Exception { assertEquals(qryId0, qryId); assertFalse(res.get("last").asBoolean()); - ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.FETCH_SQL_QUERY.key(), - "pageSize", "1", "qryId", qryId.asText())); + ret = content(DEFAULT_CACHE_NAME, GridRestCommand.FETCH_SQL_QUERY, + "pageSize", "1", + "qryId", qryId.asText() + ); res = jsonResponse(ret); @@ -1826,16 +2002,13 @@ public void testDistributedJoinsQuery() throws Exception { "where \"organization\".Organization.id = Person.orgId " + "and \"organization\".Organization.name = ?"; - Map params = new HashMap<>(); - params.put("cmd", GridRestCommand.EXECUTE_SQL_QUERY.key()); - params.put("type", "Person"); - params.put("distributedJoins", "true"); - params.put("pageSize", "10"); - params.put("cacheName", "person"); - params.put("qry", URLEncoder.encode(qry, CHARSET)); - params.put("arg1", "o1"); - - String ret = content(params); + String ret = content("person", GridRestCommand.EXECUTE_SQL_QUERY, + "type", "Person", + "distributedJoins", "true", + "pageSize", "10", + "qry", URLEncoder.encode(qry, CHARSET), + "arg1", "o1" + ); JsonNode items = jsonResponse(ret).get("items"); @@ -1850,13 +2023,10 @@ public void testDistributedJoinsQuery() throws Exception { public void testSqlFieldsQuery() throws Exception { String qry = "select concat(firstName, ' ', lastName) from Person"; - Map params = new HashMap<>(); - params.put("cmd", GridRestCommand.EXECUTE_SQL_FIELDS_QUERY.key()); - params.put("pageSize", "10"); - params.put("cacheName", "person"); - params.put("qry", URLEncoder.encode(qry, CHARSET)); - - String ret = content(params); + String ret = content("person", GridRestCommand.EXECUTE_SQL_FIELDS_QUERY, + "pageSize", "10", + "qry", URLEncoder.encode(qry, CHARSET) + ); JsonNode items = jsonResponse(ret).get("items"); @@ -1871,14 +2041,11 @@ public void testSqlFieldsQuery() throws Exception { public void testDistributedJoinsSqlFieldsQuery() throws Exception { String qry = "select * from \"person\".Person p, \"organization\".Organization o where o.id = p.orgId"; - Map params = new HashMap<>(); - params.put("cmd", GridRestCommand.EXECUTE_SQL_FIELDS_QUERY.key()); - params.put("distributedJoins", "true"); - params.put("pageSize", "10"); - params.put("cacheName", "person"); - params.put("qry", URLEncoder.encode(qry, CHARSET)); - - String ret = content(params); + String ret = content("person", GridRestCommand.EXECUTE_SQL_FIELDS_QUERY, + "distributedJoins", "true", + "pageSize", "10", + "qry", URLEncoder.encode(qry, CHARSET) + ); JsonNode items = jsonResponse(ret).get("items"); @@ -1893,13 +2060,10 @@ public void testDistributedJoinsSqlFieldsQuery() throws Exception { public void testSqlFieldsMetadataQuery() throws Exception { String qry = "select firstName, lastName from Person"; - Map params = new HashMap<>(); - params.put("cmd", GridRestCommand.EXECUTE_SQL_FIELDS_QUERY.key()); - params.put("pageSize", "10"); - params.put("cacheName", "person"); - params.put("qry", URLEncoder.encode(qry, CHARSET)); - - String ret = content(params); + String ret = content("person", GridRestCommand.EXECUTE_SQL_FIELDS_QUERY, + "pageSize", "10", + "qry", URLEncoder.encode(qry, CHARSET) + ); JsonNode res = jsonResponse(ret); @@ -1926,16 +2090,13 @@ public void testSqlFieldsMetadataQuery() throws Exception { public void testQueryClose() throws Exception { String qry = "salary > ? and salary <= ?"; - Map params = new HashMap<>(); - params.put("cmd", GridRestCommand.EXECUTE_SQL_QUERY.key()); - params.put("type", "Person"); - params.put("pageSize", "1"); - params.put("cacheName", "person"); - params.put("qry", URLEncoder.encode(qry, CHARSET)); - params.put("arg1", "1000"); - params.put("arg2", "2000"); - - String ret = content(params); + String ret = content("person", GridRestCommand.EXECUTE_SQL_QUERY, + "type", "Person", + "pageSize", "1", + "qry", URLEncoder.encode(qry, CHARSET), + "arg1", "1000", + "arg2", "2000" + ); JsonNode res = jsonResponse(ret); @@ -1947,7 +2108,7 @@ public void testQueryClose() throws Exception { String qryId = res.get("queryId").asText(); - content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CLOSE_SQL_QUERY.key(), "cacheName", "person", "qryId", qryId)); + content("person", GridRestCommand.CLOSE_SQL_QUERY, "qryId", qryId); assertFalse(queryCursorFound()); } @@ -1958,19 +2119,16 @@ public void testQueryClose() throws Exception { public void testQueryDelay() throws Exception { String qry = "salary > ? and salary <= ?"; - Map params = new HashMap<>(); - params.put("cmd", GridRestCommand.EXECUTE_SQL_QUERY.key()); - params.put("type", "Person"); - params.put("pageSize", "1"); - params.put("cacheName", "person"); - params.put("qry", URLEncoder.encode(qry, CHARSET)); - params.put("arg1", "1000"); - params.put("arg2", "2000"); - String ret = null; for (int i = 0; i < 10; ++i) - ret = content(params); + ret = content("person", GridRestCommand.EXECUTE_SQL_QUERY, + "type", "Person", + "pageSize", "1", + "qry", URLEncoder.encode(qry, CHARSET), + "arg1", "1000", + "arg2", "2000" + ); JsonNode items = jsonResponse(ret).get("items"); @@ -2003,7 +2161,7 @@ private boolean queryCursorFound() { ConcurrentHashMap its = GridTestUtils.getFieldValue(qryHnd, "qryCurs"); - found |= its.size() != 0; + found |= !its.isEmpty(); } return found; @@ -2380,11 +2538,20 @@ private static String concat(Object[] vals, String delim) { FileSystemConfiguration igfs = new FileSystemConfiguration(); igfs.setName("igfs"); - igfs.setIpcEndpointConfiguration(new IgfsIpcEndpointConfiguration()); cfg.setFileSystemConfiguration(igfs); + DataStorageConfiguration dsCfg = new DataStorageConfiguration(); + + DataRegionConfiguration drCfg = new DataRegionConfiguration(); + drCfg.setName("testDataRegion"); + drCfg.setMaxSize(100 * 1024 * 1024); + + dsCfg.setDefaultDataRegionConfiguration(drCfg); + + cfg.setDataStorageConfiguration(dsCfg); + return cfg; } @@ -2395,13 +2562,13 @@ private static String concat(Object[] vals, String delim) { * @throws Exception If failed. */ private void assertClusterState(boolean exp) throws Exception { - String ret = content(F.asMap("cmd", GridRestCommand.CLUSTER_CURRENT_STATE.key())); + String ret = content("cmd", GridRestCommand.CLUSTER_CURRENT_STATE); info("Cluster state: " + ret); JsonNode res = jsonResponse(ret); assertEquals(exp, res.asBoolean()); - assertEquals(exp, grid(0).active()); + assertEquals(exp, grid(0).cluster().active()); } /** @@ -2411,9 +2578,9 @@ private void assertClusterState(boolean exp) throws Exception { * @throws Exception If failed. */ private void changeClusterState(boolean state) throws Exception { - String cmd = (state ? GridRestCommand.CLUSTER_ACTIVE : GridRestCommand.CLUSTER_INACTIVE).key(); + GridRestCommand cmd = state ? GridRestCommand.CLUSTER_ACTIVE : GridRestCommand.CLUSTER_INACTIVE; - String ret = content(F.asMap("cmd", cmd)); + String ret = content(null, cmd); JsonNode res = jsonResponse(ret); 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 30949632ae637..671654224a486 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 @@ -118,6 +118,7 @@ import org.apache.ignite.internal.marshaller.optimized.OptimizedMarshaller; import org.apache.ignite.internal.processors.GridProcessor; import org.apache.ignite.internal.processors.affinity.GridAffinityProcessor; +import org.apache.ignite.internal.processors.cache.CacheConfigurationOverride; import org.apache.ignite.internal.processors.cache.GridCacheAdapter; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheProcessor; @@ -443,14 +444,12 @@ public IgniteKernal(@Nullable GridSpringResourceContext rsrcCtx) { } /** {@inheritDoc} */ - @Override - public boolean isRebalanceEnabled() { + @Override public boolean isRebalanceEnabled() { return ctx.cache().context().isRebalanceEnabled(); } /** {@inheritDoc} */ - @Override - public void rebalanceEnabled(boolean rebalanceEnabled) { + @Override public void rebalanceEnabled(boolean rebalanceEnabled) { ctx.cache().context().rebalanceEnabled(rebalanceEnabled); } @@ -3163,10 +3162,13 @@ public IgniteInternalFuture destroyCachesAsync(Collection cacheNames, /** * @param cacheName Cache name. + * @param templateName Template name. + * @param cfgOverride Cache config properties to override. * @param checkThreadTx If {@code true} checks that current thread does not have active transactions. * @return Future that will be completed when cache is deployed. */ - public IgniteInternalFuture getOrCreateCacheAsync(String cacheName, boolean checkThreadTx) { + public IgniteInternalFuture getOrCreateCacheAsync(String cacheName, String templateName, + CacheConfigurationOverride cfgOverride, boolean checkThreadTx) { CU.validateCacheName(cacheName); guard(); @@ -3175,7 +3177,7 @@ public IgniteInternalFuture getOrCreateCacheAsync(String cacheName, boolean c checkClusterState(); if (ctx.cache().cache(cacheName) == null) - return ctx.cache().getOrCreateFromTemplate(cacheName, checkThreadTx); + return ctx.cache().getOrCreateFromTemplate(cacheName, templateName, cfgOverride, checkThreadTx); return new GridFinishedFuture<>(); } @@ -4010,7 +4012,7 @@ private boolean skipDaemon(GridComponent comp) { } /** {@inheritDoc} */ - public void dumpDebugInfo() { + @Override public void dumpDebugInfo() { try { GridKernalContextImpl ctx = this.ctx; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheConfigurationOverride.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheConfigurationOverride.java new file mode 100644 index 0000000000000..7bd607229f88d --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheConfigurationOverride.java @@ -0,0 +1,165 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.CacheMode; +import org.apache.ignite.cache.CacheWriteSynchronizationMode; +import org.apache.ignite.configuration.CacheConfiguration; + +/** + * Helper class to override cache configuration. + */ +public class CacheConfigurationOverride { + /** */ + private CacheMode mode; + + /** */ + private Integer backups; + + /** */ + private String cacheGroup; + + /** */ + private String dataRegion; + + /** */ + private CacheWriteSynchronizationMode writeSync; + + /** + * @return Cache mode. + */ + public CacheMode mode() { + return mode; + } + + /** + * @param mode New cache mode. + * @return {@code this} for chaining. + */ + public CacheConfigurationOverride mode(CacheMode mode) { + this.mode = mode; + + return this; + } + + /** + * @return Number of backup nodes for one partition. + */ + public Integer backups() { + return backups; + } + + /** + * @param backups New number of backup nodes for one partition. + * @return {@code this} for chaining. + */ + public CacheConfigurationOverride backups(Integer backups) { + this.backups = backups; + + return this; + } + + /** + * @return Cache group name. + */ + public String cacheGroup() { + return cacheGroup; + } + + /** + * @param grpName New cache group name. + * @return {@code this} for chaining. + */ + public CacheConfigurationOverride cacheGroup(String grpName) { + this.cacheGroup = grpName; + + return this; + } + + /** + * @return Data region name. + */ + public String dataRegion() { + return dataRegion; + } + + /** + * @param dataRegName Data region name. + * @return {@code this} for chaining. + */ + public CacheConfigurationOverride dataRegion(String dataRegName) { + this.dataRegion = dataRegName; + + return this; + } + + /** + * @return Write synchronization mode. + */ + public CacheWriteSynchronizationMode writeSynchronizationMode() { + return writeSync; + } + + /** + * @param writeSync New write synchronization mode. + * @return {@code this} for chaining. + */ + public CacheConfigurationOverride writeSynchronizationMode(CacheWriteSynchronizationMode writeSync) { + this.writeSync = writeSync; + + return this; + } + + /** + * Apply overrides to specified cache configuration. + * + * @param ccfg Cache configuration to override. + * @return Updated cache configuration to permit fluent-style method calls. + */ + public CacheConfiguration apply(CacheConfiguration ccfg) { + assert ccfg != null; + + if (mode != null) + ccfg.setCacheMode(mode); + + if (backups != null) + ccfg.setBackups(backups); + + if (cacheGroup != null) + ccfg.setGroupName(cacheGroup); + + if (dataRegion != null) + ccfg.setDataRegionName(dataRegion); + + if (writeSync != null) + ccfg.setWriteSynchronizationMode(writeSync); + + return ccfg; + } + + /** + * @return {@code true} If nothing was set. + */ + public boolean isEmpty() { + return mode == null && + backups == null && + cacheGroup == null && + dataRegion == null && + writeSync == null; + } +} 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 94a9037b6a55a..8db4567c5e221 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java @@ -2574,15 +2574,36 @@ public IgniteInternalFuture createFromTemplate(String cacheName) { * @return Future that will be completed when cache is deployed. */ public IgniteInternalFuture getOrCreateFromTemplate(String cacheName, boolean checkThreadTx) { + return getOrCreateFromTemplate(cacheName, cacheName, null, checkThreadTx); + } + + /** + * Dynamically starts cache using template configuration. + * + * @param cacheName Cache name. + * @param templateName Cache template name. + * @param cfgOverride Cache config properties to override. + * @param checkThreadTx If {@code true} checks that current thread does not have active transactions. + * @return Future that will be completed when cache is deployed. + */ + public IgniteInternalFuture getOrCreateFromTemplate(String cacheName, String templateName, + CacheConfigurationOverride cfgOverride, boolean checkThreadTx) { assert cacheName != null; try { if (publicJCache(cacheName, false, checkThreadTx) != null) // Cache with given name already started. return new GridFinishedFuture<>(); - CacheConfiguration cfg = getOrCreateConfigFromTemplate(cacheName); + CacheConfiguration ccfg = F.isEmpty(templateName) + ? getOrCreateConfigFromTemplate(cacheName) + : getOrCreateConfigFromTemplate(templateName); + + ccfg.setName(cacheName); + + if (cfgOverride != null) + cfgOverride.apply(ccfg); - return dynamicStartCache(cfg, cacheName, null, false, true, checkThreadTx); + return dynamicStartCache(ccfg, cacheName, null, false, true, checkThreadTx); } catch (IgniteCheckedException e) { return new GridFinishedFuture<>(e); @@ -3083,6 +3104,10 @@ public IgniteInternalFuture resetCacheState(Collection cacheNames) { return fut; } + /** + * @param cacheName Cache name. + * @return Cache type. + */ public CacheType cacheType(String cacheName ) { if (CU.isUtilityCache(cacheName)) return CacheType.UTILITY; 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 0d63fc0dc66b5..8a7fb416228fb 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 @@ -1158,7 +1158,7 @@ public static long toExpireTime(long ttl) { public static void inTx(IgniteInternalCache cache, TransactionConcurrency concurrency, TransactionIsolation isolation, IgniteInClosureX> clo) throws IgniteCheckedException { - try (GridNearTxLocal tx = cache.txStartEx(concurrency, isolation);) { + try (GridNearTxLocal tx = cache.txStartEx(concurrency, isolation)) { clo.applyx(cache); tx.commit(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java index 21bab223a8634..f9f2cc3109ec0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java @@ -39,7 +39,6 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.cache.CacheMetrics; -import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cache.CachePeekMode; import org.apache.ignite.cluster.ClusterGroup; import org.apache.ignite.cluster.ClusterNode; @@ -52,6 +51,7 @@ import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteKernal; import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager; +import org.apache.ignite.internal.processors.cache.CacheConfigurationOverride; import org.apache.ignite.internal.processors.cache.CacheInvokeEntry; import org.apache.ignite.internal.processors.cache.GridCacheAdapter; import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; @@ -78,7 +78,12 @@ import org.jetbrains.annotations.Nullable; import static java.util.concurrent.TimeUnit.MILLISECONDS; +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; import static org.apache.ignite.internal.GridClosureCallMode.BALANCE; +import static org.apache.ignite.internal.processors.query.QueryUtils.TEMPLATE_PARTITIONED; +import static org.apache.ignite.internal.processors.query.QueryUtils.TEMPLATE_REPLICATED; import static org.apache.ignite.internal.processors.rest.GridRestCommand.ATOMIC_DECREMENT; import static org.apache.ignite.internal.processors.rest.GridRestCommand.ATOMIC_INCREMENT; import static org.apache.ignite.internal.processors.rest.GridRestCommand.CACHE_ADD; @@ -87,7 +92,6 @@ import static org.apache.ignite.internal.processors.rest.GridRestCommand.CACHE_CLEAR; import static org.apache.ignite.internal.processors.rest.GridRestCommand.CACHE_CONTAINS_KEY; import static org.apache.ignite.internal.processors.rest.GridRestCommand.CACHE_CONTAINS_KEYS; -import static org.apache.ignite.internal.processors.rest.GridRestCommand.CACHE_UPDATE_TLL; import static org.apache.ignite.internal.processors.rest.GridRestCommand.CACHE_GET; import static org.apache.ignite.internal.processors.rest.GridRestCommand.CACHE_GET_ALL; import static org.apache.ignite.internal.processors.rest.GridRestCommand.CACHE_GET_AND_PUT; @@ -106,6 +110,7 @@ import static org.apache.ignite.internal.processors.rest.GridRestCommand.CACHE_REPLACE; import static org.apache.ignite.internal.processors.rest.GridRestCommand.CACHE_REPLACE_VALUE; import static org.apache.ignite.internal.processors.rest.GridRestCommand.CACHE_SIZE; +import static org.apache.ignite.internal.processors.rest.GridRestCommand.CACHE_UPDATE_TLL; import static org.apache.ignite.internal.processors.rest.GridRestCommand.DESTROY_CACHE; import static org.apache.ignite.internal.processors.rest.GridRestCommand.GET_OR_CREATE_CACHE; import static org.apache.ignite.internal.processors.task.GridTaskThreadContextKey.TC_NO_FAILOVER; @@ -217,11 +222,9 @@ private static IgniteInternalFuture appendOrPrepend( throw new IgniteCheckedException(GridRestCommandHandlerAdapter.missingParameter("val")); return ctx.closure().callLocalSafe(new Callable() { - @Override - public Object call() throws Exception { + @Override public Object call() throws Exception { EntryProcessorResult res = cache.invoke(key, new EntryProcessor() { - @Override - public Boolean process(MutableEntry entry, + @Override public Boolean process(MutableEntry entry, Object... objects) throws EntryProcessorException { try { Object curVal = entry.getValue(); @@ -402,9 +405,29 @@ private static IgniteInternalCache cache(Ignite ignite, } case GET_OR_CREATE_CACHE: { + String templateName = req0.templateName(); + + if (F.isEmpty(templateName)) + templateName = TEMPLATE_PARTITIONED; + + CacheConfigurationOverride cfgOverride = req0.configuration(); + + boolean dfltPartTemplate = F.isEmpty(templateName) || TEMPLATE_PARTITIONED.equalsIgnoreCase(templateName); + boolean dfltReplTemplate = TEMPLATE_REPLICATED.equalsIgnoreCase(templateName); + + if (dfltPartTemplate || dfltReplTemplate) { + if (cfgOverride == null) + cfgOverride = new CacheConfigurationOverride(); + + cfgOverride.mode(dfltPartTemplate ? PARTITIONED : REPLICATED); + + if (cfgOverride.writeSynchronizationMode() == null) + cfgOverride.writeSynchronizationMode(FULL_SYNC); + } + // Do not check thread tx here since there can be active system cache txs. - fut = ((IgniteKernal)ctx.grid()).getOrCreateCacheAsync(cacheName, false).chain( - new CX1, GridRestResponse>() { + fut = ((IgniteKernal)ctx.grid()).getOrCreateCacheAsync(cacheName, templateName, cfgOverride, false) + .chain(new CX1, GridRestResponse>() { @Override public GridRestResponse applyx(IgniteInternalFuture f) throws IgniteCheckedException { f.get(); @@ -800,7 +823,7 @@ private IgniteInternalFuture executeCommand( private boolean replicatedCacheAvailable(String cacheName) { GridCacheAdapter cache = ctx.cache().internalCache(cacheName); - return cache != null && cache.configuration().getCacheMode() == CacheMode.REPLICATED; + return cache != null && cache.configuration().getCacheMode() == REPLICATED; } /** @@ -1080,7 +1103,7 @@ private static class MetadataJob extends ComputeJobAdapter { @Override public Collection execute() { String cacheName = null; - if (!ignite.active()) + if (!ignite.cluster().active()) return Collections.emptyList(); IgniteInternalCache cache = null; @@ -1673,8 +1696,7 @@ private static class UpdateTllCommand extends CacheCommand { return ctx.closure().callLocalSafe(new Callable() { @Override public Object call() throws Exception { EntryProcessorResult res = c.invoke(key, new EntryProcessor() { - @Override - public Boolean process(MutableEntry entry, + @Override public Boolean process(MutableEntry entry, Object... objects) throws EntryProcessorException { GridCacheEntryEx ex = ((CacheInvokeEntry)entry).entry(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/GridRestCacheRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/GridRestCacheRequest.java index b9320365e4183..7e1924528d9a5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/GridRestCacheRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/GridRestCacheRequest.java @@ -18,15 +18,19 @@ package org.apache.ignite.internal.processors.rest.request; import java.util.Map; +import org.apache.ignite.internal.processors.cache.CacheConfigurationOverride; import org.apache.ignite.internal.util.typedef.internal.S; /** - * + * Cache command request descriptor. */ public class GridRestCacheRequest extends GridRestRequest { /** Cache name. */ private String cacheName; + /** Template name. */ + private String templateName; + /** Key. */ private Object key; @@ -39,6 +43,9 @@ public class GridRestCacheRequest extends GridRestRequest { /** Keys and values for put all, get all, remove all operations. */ private Map vals; + /** Cache configuration parameters. */ + private CacheConfigurationOverride cfg; + /** Bit map of cache flags to be enabled on cache projection. */ private int cacheFlags; @@ -59,6 +66,20 @@ public void cacheName(String cacheName) { this.cacheName = cacheName; } + /** + * @return Template name, or {@code null} if not set. + */ + public String templateName() { + return templateName; + } + + /** + * @param templateName Template name. + */ + public void templateName(String templateName) { + this.templateName = templateName; + } + /** * @return Key. */ @@ -115,6 +136,21 @@ public void values(Map vals) { this.vals = vals; } + + /** + * @return Cache configuration. + */ + public CacheConfigurationOverride configuration() { + return cfg; + } + + /** + * @param cfg Cache configuration. + */ + public void configuration(CacheConfigurationOverride cfg) { + this.cfg = cfg; + } + /** * @param cacheFlags Bit representation of cache flags. */ @@ -147,4 +183,4 @@ public void ttl(Long ttl) { @Override public String toString() { return S.toString(GridRestCacheRequest.class, this, super.toString()); } -} \ No newline at end of file +} diff --git a/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/GridJettyRestHandler.java b/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/GridJettyRestHandler.java index b7f0d453da58d..7bad10d78dcf6 100644 --- a/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/GridJettyRestHandler.java +++ b/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/GridJettyRestHandler.java @@ -17,8 +17,6 @@ package org.apache.ignite.internal.processors.rest.protocols.http.jetty; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectMapper; import java.io.BufferedInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -33,13 +31,15 @@ import java.util.List; import java.util.Map; import java.util.UUID; -import javax.servlet.ServletException; import javax.servlet.ServletOutputStream; import javax.servlet.ServletRequest; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; + import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.cache.CacheWriteSynchronizationMode; +import org.apache.ignite.internal.processors.cache.CacheConfigurationOverride; import org.apache.ignite.internal.processors.rest.GridRestCommand; import org.apache.ignite.internal.processors.rest.GridRestProtocolHandler; import org.apache.ignite.internal.processors.rest.GridRestResponse; @@ -55,8 +55,11 @@ import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteClosure; import org.apache.ignite.plugin.security.SecurityCredentials; + import org.eclipse.jetty.server.Request; import org.eclipse.jetty.server.handler.AbstractHandler; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; import org.jetbrains.annotations.Nullable; import static org.apache.ignite.internal.processors.rest.GridRestCommand.CACHE_CONTAINS_KEYS; @@ -75,6 +78,24 @@ public class GridJettyRestHandler extends AbstractHandler { /** Used to sent request charset. */ private static final String CHARSET = StandardCharsets.UTF_8.name(); + /** */ + private static final String CACHE_NAME_PARAM = "cacheName"; + + /** */ + private static final String BACKUPS_PARAM = "backups"; + + /** */ + private static final String CACHE_GROUP_PARAM = "cacheGroup"; + + /** */ + private static final String DATA_REGION_PARAM = "dataRegion"; + + /** */ + private static final String WRITE_SYNCHRONIZATION_MODE_PARAM = "writeSynchronizationMode"; + + /** */ + private static final String TEMPLATE_NAME_PARAM = "templateName"; + /** Logger. */ private final IgniteLogger log; @@ -261,7 +282,7 @@ private void initFavicon() throws IOException { /** {@inheritDoc} */ @Override public void handle(String target, Request req, HttpServletRequest srvReq, HttpServletResponse res) - throws IOException, ServletException { + throws IOException { if (log.isDebugEnabled()) log.debug("Handling request [target=" + target + ", req=" + req + ", srvReq=" + srvReq + ']'); @@ -400,11 +421,66 @@ private void processRequest(String act, HttpServletRequest req, HttpServletRespo GridRestRequest restReq; switch (cmd) { - case GET_OR_CREATE_CACHE: + case GET_OR_CREATE_CACHE: { + GridRestCacheRequest restReq0 = new GridRestCacheRequest(); + + restReq0.cacheName((String)params.get(CACHE_NAME_PARAM)); + + String templateName = (String)params.get(TEMPLATE_NAME_PARAM); + + if (!F.isEmpty(templateName)) + restReq0.templateName(templateName); + + String backups = (String)params.get(BACKUPS_PARAM); + + CacheConfigurationOverride cfg = new CacheConfigurationOverride(); + + // Set cache backups. + if (!F.isEmpty(backups)) { + try { + cfg.backups(Integer.parseInt(backups)); + } + catch (NumberFormatException e) { + throw new IgniteCheckedException("Failed to parse number of cache backups: " + backups, e); + } + } + + // Set cache group name. + String cacheGroup = (String)params.get(CACHE_GROUP_PARAM); + + if (!F.isEmpty(cacheGroup)) + cfg.cacheGroup(cacheGroup); + + // Set cache data region name. + String dataRegion = (String)params.get(DATA_REGION_PARAM); + + if (!F.isEmpty(dataRegion)) + cfg.dataRegion(dataRegion); + + // Set cache write mode. + String wrtSyncMode = (String)params.get(WRITE_SYNCHRONIZATION_MODE_PARAM); + + if (!F.isEmpty(wrtSyncMode)) { + try { + cfg.writeSynchronizationMode(CacheWriteSynchronizationMode.valueOf(wrtSyncMode)); + } + catch (IllegalArgumentException e) { + throw new IgniteCheckedException("Failed to parse cache write synchronization mode: " + wrtSyncMode, e); + } + } + + if (!cfg.isEmpty()) + restReq0.configuration(cfg); + + restReq = restReq0; + + break; + } + case DESTROY_CACHE: { GridRestCacheRequest restReq0 = new GridRestCacheRequest(); - restReq0.cacheName((String)params.get("cacheName")); + restReq0.cacheName((String)params.get(CACHE_NAME_PARAM)); restReq = restReq0; @@ -450,7 +526,7 @@ private void processRequest(String act, HttpServletRequest req, HttpServletRespo case CACHE_PREPEND: { GridRestCacheRequest restReq0 = new GridRestCacheRequest(); - String cacheName = (String)params.get("cacheName"); + String cacheName = (String)params.get(CACHE_NAME_PARAM); restReq0.cacheName(F.isEmpty(cacheName) ? null : cacheName); restReq0.key(params.get("key")); @@ -579,7 +655,7 @@ private void processRequest(String act, HttpServletRequest req, HttpServletRespo if (distributedJoins != null) restReq0.distributedJoins(Boolean.parseBoolean(distributedJoins)); - restReq0.cacheName((String)params.get("cacheName")); + restReq0.cacheName((String)params.get(CACHE_NAME_PARAM)); if (cmd == EXECUTE_SQL_QUERY) restReq0.queryType(RestQueryRequest.QueryType.SQL); @@ -601,7 +677,7 @@ private void processRequest(String act, HttpServletRequest req, HttpServletRespo if (pageSize != null) restReq0.pageSize(Integer.parseInt(pageSize)); - restReq0.cacheName((String)params.get("cacheName")); + restReq0.cacheName((String)params.get(CACHE_NAME_PARAM)); restReq0.className((String)params.get("className")); @@ -625,7 +701,7 @@ private void processRequest(String act, HttpServletRequest req, HttpServletRespo if (pageSize != null) restReq0.pageSize(Integer.parseInt(pageSize)); - restReq0.cacheName((String)params.get("cacheName")); + restReq0.cacheName((String)params.get(CACHE_NAME_PARAM)); restReq = restReq0; @@ -640,7 +716,7 @@ private void processRequest(String act, HttpServletRequest req, HttpServletRespo if (qryId != null) restReq0.queryId(Long.parseLong(qryId)); - restReq0.cacheName((String)params.get("cacheName")); + restReq0.cacheName((String)params.get(CACHE_NAME_PARAM)); restReq = restReq0; @@ -757,7 +833,8 @@ private Map parameters(ServletRequest req) { @Nullable private String parameter(Object obj) { if (obj instanceof String) return (String)obj; - else if (obj instanceof String[] && ((String[])obj).length > 0) + + if (obj instanceof String[] && ((String[])obj).length > 0) return ((String[])obj)[0]; return null; From 6cb0052a5ece8e1bac3c68e4c0986c22c489e760 Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Tue, 20 Feb 2018 10:32:27 +0700 Subject: [PATCH 028/314] IGNITE-3345 REST: Added support for Java built in types (boolean, int, long, ..., UUID) for put/get operations. (cherry picked from commit 3ba73ca) --- .../JettyRestProcessorAbstractSelfTest.java | 342 ++++++++++++++++++ .../http/jetty/GridJettyRestHandler.java | 105 +++++- 2 files changed, 436 insertions(+), 11 deletions(-) diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java index a33f0199eaa3f..e4433cf192c23 100644 --- a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java @@ -27,6 +27,9 @@ import java.net.URLConnection; import java.net.URLEncoder; import java.nio.charset.StandardCharsets; +import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; import java.text.DateFormat; import java.util.Collection; import java.util.HashMap; @@ -152,6 +155,8 @@ import static org.apache.ignite.internal.IgniteVersionUtils.VER_STR; import static org.apache.ignite.internal.processors.query.QueryUtils.TEMPLATE_PARTITIONED; import static org.apache.ignite.internal.processors.query.QueryUtils.TEMPLATE_REPLICATED; +import static org.apache.ignite.internal.processors.rest.GridRestResponse.STATUS_FAILED; +import static org.apache.ignite.internal.processors.rest.GridRestResponse.STATUS_SUCCESS; /** * Tests for Jetty REST protocol. @@ -2141,6 +2146,340 @@ public void testQueryDelay() throws Exception { assertFalse(queryCursorFound()); } + /** + * @return Cache. + */ + protected IgniteCache typedCache() { + return grid(0).cache("test_typed_access"); + } + + /** + * @param type Key and value type. + * @param k Key to put. + * @param v Value to put. + * @param status Expected operation status to check. + * @throws Exception If failed. + */ + private void putTypedValue(String type, String k, String v, int status) throws Exception { + String ret = content("test_typed_access", GridRestCommand.CACHE_PUT, + "keyType", type, + "valueType", type, + "key", k, + "val", v + ); + + info("Command result: " + ret); + + JsonNode json = JSON_MAPPER.readTree(ret); + + assertEquals(status, json.get("successStatus").asInt()); + + if (status == STATUS_SUCCESS) + assertTrue(json.get("error").isNull()); + else + assertTrue(json.get("error").asText().startsWith("Failed to convert value to specified type [type=")); + } + + /** + * @throws Exception If failed. + */ + public void testTypedPut() throws Exception { + // Test boolean type. + putTypedValue("boolean", "true", "false", STATUS_SUCCESS); + putTypedValue("java.lang.Boolean", "false", "true", STATUS_SUCCESS); + + IgniteCache cBool = typedCache(); + + assertEquals(cBool.get(true), Boolean.FALSE); + assertEquals(cBool.get(false), Boolean.TRUE); + + // Test byte type. + putTypedValue("byte", "64", "100", STATUS_SUCCESS); + putTypedValue("java.lang.Byte", "-25", "-127", STATUS_SUCCESS); + putTypedValue("byte", "65", "aaa", STATUS_FAILED); + putTypedValue("byte", "aaa", "64", STATUS_FAILED); + putTypedValue("byte", "aaa", "aaa", STATUS_FAILED); + + IgniteCache cByte = typedCache(); + + assertEquals(cByte.get(Byte.valueOf("64")), Byte.valueOf("100")); + assertEquals(cByte.get(Byte.valueOf("-25")), Byte.valueOf("-127")); + + // Test short type. + putTypedValue("short", "1024", "4096", STATUS_SUCCESS); + putTypedValue("java.lang.Short", "-15000", "-16000", STATUS_SUCCESS); + putTypedValue("short", "1025", "bbb", STATUS_FAILED); + putTypedValue("short", "bbb", "5", STATUS_FAILED); + putTypedValue("short", "bbb", "bbb", STATUS_FAILED); + + IgniteCache cShort = typedCache(); + + assertEquals(cShort.get(Short.valueOf("1024")), Short.valueOf("4096")); + assertEquals(cShort.get(Short.valueOf("-15000")), Short.valueOf("-16000")); + + // Test integer type. + putTypedValue("int", "65555", "128256", STATUS_SUCCESS); + putTypedValue("Integer", "74555", "200000", STATUS_SUCCESS); + putTypedValue("java.lang.Integer", "-200", "-100000", STATUS_SUCCESS); + putTypedValue("int", "0", "ccc", STATUS_FAILED); + putTypedValue("int", "ccc", "0", STATUS_FAILED); + putTypedValue("int", "ccc", "ccc", STATUS_FAILED); + + IgniteCache cInt = typedCache(); + + assertEquals(cInt.get(65555), Integer.valueOf(128256)); + assertEquals(cInt.get(74555), Integer.valueOf(200000)); + assertEquals(cInt.get(-200), Integer.valueOf(-100000)); + + // Test long type. + putTypedValue("long", "3000000", "400000", STATUS_SUCCESS); + putTypedValue("java.lang.Long", "-3000000", "-400000", STATUS_SUCCESS); + putTypedValue("long", "777", "ddd", STATUS_FAILED); + putTypedValue("long", "ddd", "777", STATUS_FAILED); + putTypedValue("long", "ddd", "ddd", STATUS_FAILED); + + IgniteCache cLong = typedCache(); + + assertEquals(cLong.get(3000000L), Long.valueOf(400000)); + assertEquals(cLong.get(-3000000L), Long.valueOf(-400000)); + + // Test float type. + putTypedValue("float", "1.5", "2.5", STATUS_SUCCESS); + putTypedValue("java.lang.Float", "-7.5", "-8.5", STATUS_SUCCESS); + putTypedValue("float", "1.5", "hhh", STATUS_FAILED); + putTypedValue("float", "hhh", "1.5", STATUS_FAILED); + putTypedValue("float", "hhh", "hhh", STATUS_FAILED); + + IgniteCache cFloat = typedCache(); + + assertEquals(cFloat.get(1.5f), 2.5f); + assertEquals(cFloat.get(-7.5f), -8.5f); + + // Test double type. + putTypedValue("double", "5.5", "75.5", STATUS_SUCCESS); + putTypedValue("java.lang.Double", "-155.5", "-255.5", STATUS_SUCCESS); + putTypedValue("double", "jjj", "75.5", STATUS_FAILED); + putTypedValue("double", "6.5", "jjj", STATUS_FAILED); + putTypedValue("double", "jjj", "jjj", STATUS_FAILED); + + IgniteCache cDouble = typedCache(); + + assertEquals(cDouble.get(5.5d), 75.5d); + assertEquals(cDouble.get(-155.5d), -255.5d); + + // Test date type. + putTypedValue("date", "2018-02-18", "2017-01-01", STATUS_SUCCESS); + putTypedValue("java.sql.Date", "2018-01-01", "2017-02-02", STATUS_SUCCESS); + putTypedValue("date", "xxxx-yy-mm", "2017-01-01", STATUS_FAILED); + putTypedValue("date", "2018-03-18", "xxxx-yy-mm", STATUS_FAILED); + putTypedValue("date", "xxxx-yy-mm", "xxxx-yy-mm", STATUS_FAILED); + + IgniteCache cDate = typedCache(); + + assertEquals(cDate.get(Date.valueOf("2018-02-18")), Date.valueOf("2017-01-01")); + assertEquals(cDate.get(Date.valueOf("2018-01-01")), Date.valueOf("2017-02-02")); + + // Test time type. + putTypedValue("Time", "01:01:01", "02:02:02", STATUS_SUCCESS); + putTypedValue("java.sql.Time", "03:03:03", "04:04:04", STATUS_SUCCESS); + putTypedValue("Time", "aa:bb:dd", "02:02:02", STATUS_FAILED); + putTypedValue("Time", "01:01:01", "zz:vv:pp", STATUS_FAILED); + putTypedValue("Time", "zz:zz:zz", "zz:zz:zz", STATUS_FAILED); + + IgniteCache cTime = typedCache(); + + assertEquals(cTime.get(Time.valueOf("01:01:01")), Time.valueOf("02:02:02")); + assertEquals(cTime.get(Time.valueOf("03:03:03")), Time.valueOf("04:04:04")); + + // Test timestamp type. + putTypedValue("Timestamp", "2018-02-18%2001:01:01", "2017-01-01%2002:02:02", STATUS_SUCCESS); + putTypedValue("java.sql.timestamp", "2018-01-01%2001:01:01", "2018-05-05%2005:05:05", STATUS_SUCCESS); + putTypedValue("timestamp", "error", "2018-03-18%2001:01:01", STATUS_FAILED); + putTypedValue("timestamp", "2018-03-18%2001:01:01", "error", STATUS_FAILED); + putTypedValue("timestamp", "error", "error", STATUS_FAILED); + + IgniteCache cTimestamp = typedCache(); + + assertEquals(cTimestamp.get(Timestamp.valueOf("2018-02-18 01:01:01")), Timestamp.valueOf("2017-01-01 02:02:02")); + assertEquals(cTimestamp.get(Timestamp.valueOf("2018-01-01 01:01:01")), Timestamp.valueOf("2018-05-05 05:05:05")); + + // Test UUID type. + UUID k1 = UUID.fromString("121f5ae8-148d-11e8-b642-0ed5f89f718b"); + UUID v1 = UUID.fromString("64c6c225-b31c-4000-b136-ef14562ac785"); + putTypedValue("UUID", k1.toString(), v1.toString(), STATUS_SUCCESS); + putTypedValue("UUID", "error", v1.toString(), STATUS_FAILED); + putTypedValue("UUID", k1.toString(), "error", STATUS_FAILED); + putTypedValue("UUID", "error", "error", STATUS_FAILED); + + UUID k2 = UUID.randomUUID(); + UUID v2 = UUID.randomUUID(); + putTypedValue("java.util.UUID", k2.toString(), v2.toString(), STATUS_SUCCESS); + + IgniteCache cUUID = typedCache(); + + assertEquals(cUUID.get(k1), v1); + assertEquals(cUUID.get(k2), v2); + + // Test IgniteUuid type. + IgniteUuid ik1 = IgniteUuid.randomUuid(); + IgniteUuid iv1 = IgniteUuid.randomUuid(); + putTypedValue("IgniteUuid", ik1.toString(), iv1.toString(), STATUS_SUCCESS); + putTypedValue("IgniteUuid", "error", iv1.toString(), STATUS_FAILED); + putTypedValue("IgniteUuid", ik1.toString(), "error", STATUS_FAILED); + putTypedValue("IgniteUuid", "error", "error", STATUS_FAILED); + + IgniteUuid ik2 = IgniteUuid.randomUuid(); + IgniteUuid iv2 = IgniteUuid.randomUuid(); + putTypedValue("org.apache.ignite.lang.IgniteUuid", ik2.toString(), iv2.toString(), STATUS_SUCCESS); + + IgniteCache cIgniteUUID = typedCache(); + + assertEquals(cIgniteUUID.get(ik1), iv1); + assertEquals(cIgniteUUID.get(ik2), iv2); + } + + /** + * @param keyType Key type. + * @param k Key value. + * @param exp Expected value to test. + * @throws Exception If failed. + */ + private void getTypedValue(String keyType, String k, String exp) throws Exception { + String ret = content("test_typed_access", GridRestCommand.CACHE_GET, + "keyType", keyType, + "key", k + ); + + info("Command result: " + ret); + + assertEquals(exp, jsonResponse(ret).asText()); + } + + /** + * @throws Exception If failed. + */ + public void testTypedGet() throws Exception { + // Test boolean type. + IgniteCache cBool = typedCache(); + + cBool.put(true, false); + cBool.put(false, true); + + getTypedValue("boolean", "true", "false"); + getTypedValue("java.lang.Boolean", "false", "true"); + + // Test byte type. + IgniteCache cByte = typedCache(); + + cByte.put((byte)77, (byte)55); + cByte.put((byte)-88, (byte)-10); + + getTypedValue("byte", "77", "55"); + getTypedValue("java.lang.Byte", "-88", "-10"); + + // Test short type. + IgniteCache cShort = typedCache(); + + cShort.put((short)2222, (short)3333); + cShort.put((short)-11111, (short)-12222); + + getTypedValue("short", "2222", "3333"); + getTypedValue("java.lang.Short", "-11111", "-12222"); + + // Test integer type. + IgniteCache cInt = typedCache(); + cInt.put(65555, 128256); + cInt.put(74555, 200000); + cInt.put(-200, -100000); + + getTypedValue("int", "65555", "128256"); + getTypedValue("Integer", "74555", "200000"); + getTypedValue("java.lang.Integer", "-200", "-100000"); + + // Test long type. + IgniteCache cLong = typedCache(); + + cLong.put(3333333L, 4444444L); + cLong.put(-3333333L, -4444444L); + + getTypedValue("long", "3333333", "4444444"); + getTypedValue("java.lang.Long", "-3333333", "-4444444"); + + // Test float type. + IgniteCache cFloat = typedCache(); + + cFloat.put(11.5f, 21.5f); + cFloat.put(-71.5f, -81.5f); + + getTypedValue("float", "11.5", "21.5"); + getTypedValue("java.lang.Float", "-71.5", "-81.5"); + + // Test double type. + IgniteCache cDouble = typedCache(); + + cDouble.put(58.5d, 758.5d); + cDouble.put(-1558.5d, -2558.5d); + + getTypedValue("double", "58.5", "758.5"); + getTypedValue("java.lang.Double", "-1558.5", "-2558.5"); + + // Test date type. + IgniteCache cDate = typedCache(); + + cDate.put(Date.valueOf("2018-02-18"), Date.valueOf("2017-01-01")); + cDate.put(Date.valueOf("2018-01-01"), Date.valueOf("2017-02-02")); + + getTypedValue("Date", "2018-02-18", "2017-01-01"); + getTypedValue("java.sql.Date", "2018-01-01", "2017-02-02"); + + // Test time type. + IgniteCache cTime = typedCache(); + + cTime.put(Time.valueOf("01:01:01"), Time.valueOf("02:02:02")); + cTime.put(Time.valueOf("03:03:03"), Time.valueOf("04:04:04")); + + getTypedValue("Time", "01:01:01", "02:02:02"); + getTypedValue("java.sql.Time", "03:03:03", "04:04:04"); + + // Test timestamp type. + IgniteCache cTimestamp = typedCache(); + + cTimestamp.put(Timestamp.valueOf("2018-02-18 01:01:01"), "test1"); + cTimestamp.put(Timestamp.valueOf("2018-01-01 01:01:01"), "test2"); + + getTypedValue("Timestamp", "2018-02-18%2001:01:01", "test1"); + getTypedValue("java.sql.timestamp", "2018-01-01%2001:01:01", "test2"); + + // Test UUID type. + IgniteCache cUUID = typedCache(); + + UUID k1 = UUID.fromString("121f5ae8-148d-11e8-b642-0ed5f89f718b"); + UUID v1 = UUID.fromString("64c6c225-b31c-4000-b136-ef14562ac785"); + cUUID.put(k1, v1); + + UUID k2 = UUID.randomUUID(); + UUID v2 = UUID.randomUUID(); + cUUID.put(k2, v2); + + getTypedValue("UUID", k1.toString(), v1.toString()); + getTypedValue("java.util.UUID", k2.toString(), v2.toString()); + + + // Test IgniteUuid type. + IgniteCache cIgniteUUID = typedCache(); + + IgniteUuid ik1 = IgniteUuid.randomUuid(); + IgniteUuid iv1 = IgniteUuid.randomUuid(); + cIgniteUUID.put(ik1, iv1); + + IgniteUuid ik2 = IgniteUuid.randomUuid(); + IgniteUuid iv2 = IgniteUuid.randomUuid(); + cIgniteUUID.put(ik2, iv2); + + getTypedValue("IgniteUuid", ik1.toString(), iv1.toString()); + getTypedValue("org.apache.ignite.lang.IgniteUuid", ik2.toString(), iv2.toString()); + } + /** * @return Signature. * @throws Exception If failed. @@ -2171,6 +2510,9 @@ private boolean queryCursorFound() { * Init cache. */ protected void initCache() { + CacheConfiguration typedCache = new CacheConfiguration<>("test_typed_access"); + ignite(0).getOrCreateCache(typedCache); + CacheConfiguration orgCacheCfg = new CacheConfiguration<>("organization"); orgCacheCfg.setIndexedTypes(Integer.class, Organization.class); diff --git a/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/GridJettyRestHandler.java b/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/GridJettyRestHandler.java index 7bad10d78dcf6..0b24998899f06 100644 --- a/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/GridJettyRestHandler.java +++ b/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/GridJettyRestHandler.java @@ -25,6 +25,9 @@ import java.io.LineNumberReader; import java.net.InetSocketAddress; import java.nio.charset.StandardCharsets; +import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; import java.util.Collections; import java.util.Iterator; import java.util.LinkedList; @@ -54,6 +57,7 @@ import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteClosure; +import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.plugin.security.SecurityCredentials; import org.eclipse.jetty.server.Request; @@ -407,6 +411,81 @@ private void processRequest(String act, HttpServletRequest req, HttpServletRespo } } + /** + * @param type Optional value type. + * @param obj Object to convert. + * @return Converted value. + * @throws IgniteCheckedException If failed to convert. + */ + private Object convert(String type, Object obj) throws IgniteCheckedException { + if (F.isEmpty(type) || obj == null) + return obj; + + String s = (String)obj; + + try { + switch (type.toLowerCase()) { + case "boolean": + case "java.lang.boolean": + return Boolean.valueOf(s); + + case "byte": + case "java.lang.byte": + return Byte.valueOf(s); + + case "short": + case "java.lang.short": + return Short.valueOf(s); + + case "int": + case "integer": + case "java.lang.integer": + return Integer.valueOf(s); + + case "long": + case "java.lang.long": + return Long.valueOf(s); + + case "float": + case "java.lang.float": + return Float.valueOf(s); + + case "double": + case "java.lang.double": + return Double.valueOf(s); + + case "date": + case "java.sql.date": + return Date.valueOf(s); + + case "time": + case "java.sql.time": + return Time.valueOf(s); + + case "timestamp": + case "java.sql.timestamp": + return Timestamp.valueOf(s); + + case "uuid": + case "java.util.uuid": + return UUID.fromString(s); + + case "igniteuuid": + case "org.apache.ignite.lang.igniteuuid": + return IgniteUuid.fromString(s); + + default: + // No-op. + } + } + catch (Throwable e) { + throw new IgniteCheckedException("Failed to convert value to specified type [type=" + type + + ", val=" + s + ", reason=" + e.getClass().getName() + ": " + e.getMessage() + "]"); + } + + return obj; + } + /** * Creates REST request. * @@ -527,13 +606,16 @@ private void processRequest(String act, HttpServletRequest req, HttpServletRespo GridRestCacheRequest restReq0 = new GridRestCacheRequest(); String cacheName = (String)params.get(CACHE_NAME_PARAM); - restReq0.cacheName(F.isEmpty(cacheName) ? null : cacheName); - restReq0.key(params.get("key")); - restReq0.value(params.get("val")); - restReq0.value2(params.get("val2")); - Object val1 = params.get("val1"); + String keyType = (String)params.get("keyType"); + String valType = (String)params.get("valueType"); + + restReq0.key(convert(keyType, params.get("key"))); + restReq0.value(convert(valType, params.get("val"))); + restReq0.value2(convert(valType, params.get("val2"))); + + Object val1 = convert(valType, params.get("val1")); if (val1 != null) restReq0.value(val1); @@ -543,8 +625,8 @@ private void processRequest(String act, HttpServletRequest req, HttpServletRespo if (cmd == CACHE_GET_ALL || cmd == CACHE_PUT_ALL || cmd == CACHE_REMOVE_ALL || cmd == CACHE_CONTAINS_KEYS) { - List keys = values("k", params); - List vals = values("v", params); + List keys = values(keyType, "k", params); + List vals = values(valType, "v", params); if (keys.size() < vals.size()) throw new IgniteCheckedException("Number of keys must be greater or equals to number of values."); @@ -589,7 +671,7 @@ private void processRequest(String act, HttpServletRequest req, HttpServletRespo restReq0.taskId((String)params.get("id")); restReq0.taskName((String)params.get("name")); - restReq0.params(values("p", params)); + restReq0.params(values(null, "p", params)); restReq0.async(Boolean.parseBoolean((String)params.get("async"))); @@ -641,7 +723,7 @@ private void processRequest(String act, HttpServletRequest req, HttpServletRespo restReq0.sqlQuery((String)params.get("qry")); - restReq0.arguments(values("arg", params).toArray()); + restReq0.arguments(values(null, "arg", params).toArray()); restReq0.typeName((String)params.get("type")); @@ -774,11 +856,12 @@ private void processRequest(String act, HttpServletRequest req, HttpServletRespo /** * Gets values referenced by sequential keys, e.g. {@code key1...keyN}. * + * @param type Optional value type. * @param keyPrefix Key prefix, e.g. {@code key} for {@code key1...keyN}. * @param params Parameters map. * @return Values. */ - protected List values(String keyPrefix, Map params) { + protected List values(String type, String keyPrefix, Map params) throws IgniteCheckedException { assert keyPrefix != null; List vals = new LinkedList<>(); @@ -787,7 +870,7 @@ protected List values(String keyPrefix, Map params) { String key = keyPrefix + i; if (params.containsKey(key)) - vals.add(params.get(key)); + vals.add(convert(type, params.get(key))); else break; } From 6d43b6a31d0e7da0a1c245ed7c4ae85ea6889ddc Mon Sep 17 00:00:00 2001 From: Slava Koptilin Date: Thu, 15 Feb 2018 14:57:40 +0300 Subject: [PATCH 029/314] IGNITE-5804 ScanQuery transformer should be applied to all result pages - Fixes #3470. Signed-off-by: Alexey Goncharuk --- .../cache/query/GridCacheQueryManager.java | 287 +++++++++--------- .../GridCacheQueryTransformerSelfTest.java | 79 +++++ 2 files changed, 230 insertions(+), 136 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 fe8e0544cdcf8..0061df3927530 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 @@ -538,8 +538,8 @@ private Object unwrapIfNeeded(CacheObject obj, CacheObjectContext coctx) { * @throws IgniteCheckedException In case of error. */ @SuppressWarnings("unchecked") - private QueryResult executeQuery(GridCacheQueryAdapter qry, - @Nullable Object[] args, boolean loc, @Nullable UUID subjId, @Nullable String taskName, Object rcpt) + private QueryResult executeQuery(GridCacheQueryAdapter qry, @Nullable Object[] args, + IgniteClosure transformer, boolean loc, @Nullable UUID subjId, @Nullable String taskName, Object rcpt) throws IgniteCheckedException { if (qry.type() == null) { assert !loc; @@ -593,7 +593,7 @@ private QueryResult executeQuery(GridCacheQueryAdapter qry, taskName)); } - iter = scanIterator(qry, false); + iter = scanIterator(qry, transformer, false); break; @@ -798,12 +798,14 @@ private GridCloseableIterator> setIterator(GridCacheQueryAda /** * @param qry Query. + * @param transformer Transformer. * @param locNode Local node. * @return Full-scan row iterator. * @throws IgniteCheckedException If failed to get iterator. */ @SuppressWarnings({"unchecked"}) - private GridCloseableIterator scanIterator(final GridCacheQueryAdapter qry, boolean locNode) + private GridCloseableIterator scanIterator(final GridCacheQueryAdapter qry, IgniteClosure transformer, + boolean locNode) throws IgniteCheckedException { final IgniteBiPredicate keyValFilter = qry.scanFilter(); @@ -851,7 +853,7 @@ private GridCloseableIterator scanIterator(final GridCacheQueryAdapter qry, b it = cctx.offheap().cacheIterator(cctx.cacheId(), true, backups, topVer); } - return new ScanQueryIterator(it, qry, topVer, locPart, keyValFilter, locNode, cctx, log); + return new ScanQueryIterator(it, qry, topVer, locPart, keyValFilter, transformer, locNode, cctx, log); } catch (IgniteCheckedException | RuntimeException e) { closeScanFilter(keyValFilter); @@ -1123,11 +1125,11 @@ protected void runQuery(GridCacheQueryInfo qryInfo) { String taskName = cctx.kernalContext().task().resolveTaskName(qry.taskHash()); - IgniteSpiCloseableIterator> iter; + IgniteSpiCloseableIterator iter; GridCacheQueryType type; res = loc ? - executeQuery(qry, qryInfo.arguments(), loc, qry.subjectId(), taskName, + executeQuery(qry, qryInfo.arguments(), trans, loc, qry.subjectId(), taskName, recipient(qryInfo.senderId(), qryInfo.requestId())) : queryResult(qryInfo, taskName); @@ -1166,142 +1168,113 @@ protected void runQuery(GridCacheQueryInfo qryInfo) { if(!iter.hasNext()) break; - IgniteBiTuple row = iter.next(); + Object row0 = iter.next(); // Query is cancelled. - if (row == null) { + if (row0 == null) { onPageReady(loc, qryInfo, null, true, null); break; } - final K key = row.getKey(); - - // Filter backups for SCAN queries, if it isn't partition scan. - // Other types are filtered in indexing manager. - if (!cctx.isReplicated() && qry.type() == SCAN && qry.partition() == null && - cctx.config().getCacheMode() != LOCAL && !incBackups && - !cctx.affinity().primaryByKey(cctx.localNode(), key, topVer)) { - if (log.isDebugEnabled()) - log.debug("Ignoring backup element [row=" + row + - ", cacheMode=" + cctx.config().getCacheMode() + ", incBackups=" + incBackups + - ", primary=" + cctx.affinity().primaryByKey(cctx.localNode(), key, topVer) + ']'); - - continue; - } - - V val = row.getValue(); - - if (log.isDebugEnabled()) { - ClusterNode primaryNode = cctx.affinity().primaryByKey(key, - cctx.affinity().affinityTopologyVersion()); - - 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(S.toString("Unsuitable record value", "val", val, true)); - - continue; - } - - if (statsEnabled) { - CacheMetricsImpl metrics = cctx.cache().metrics0(); + if (type == SCAN) + // Scan iterator may return already transformed entry + data.add(row0); + else { + IgniteBiTuple row = (IgniteBiTuple) row0; - metrics.onRead(true); + final K key = row.getKey(); - metrics.addGetTimeNanos(System.nanoTime() - start); - } + V val = row.getValue(); - K key0 = null; - V val0 = null; + if (log.isDebugEnabled()) { + ClusterNode primaryNode = cctx.affinity().primaryByKey(key, + cctx.affinity().affinityTopologyVersion()); - if (readEvt && cctx.gridEvents().hasListener(EVT_CACHE_QUERY_OBJECT_READ)) { - key0 = (K)CacheObjectUtils.unwrapBinaryIfNeeded(objCtx, key, qry.keepBinary(), false); - val0 = (V)CacheObjectUtils.unwrapBinaryIfNeeded(objCtx, val, qry.keepBinary(), false); - - switch (type) { - case SQL: - cctx.gridEvents().record(new CacheQueryReadEvent<>( - cctx.localNode(), - "SQL query entry read.", - EVT_CACHE_QUERY_OBJECT_READ, - CacheQueryType.SQL.name(), - cctx.name(), - qry.queryClassName(), - qry.clause(), - null, - null, - qryInfo.arguments(), - qry.subjectId(), - taskName, - key0, - val0, - null, - null)); + 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)); + } - break; + if (val == null) { + if (log.isDebugEnabled()) + log.debug(S.toString("Unsuitable record value", "val", val, true)); - case TEXT: - cctx.gridEvents().record(new CacheQueryReadEvent<>( - cctx.localNode(), - "Full text query entry read.", - EVT_CACHE_QUERY_OBJECT_READ, - CacheQueryType.FULL_TEXT.name(), - cctx.name(), - qry.queryClassName(), - qry.clause(), - null, - null, - null, - qry.subjectId(), - taskName, - key0, - val0, - null, - null)); + continue; + } - break; + if (statsEnabled) { + CacheMetricsImpl metrics = cctx.cache().metrics0(); - case SCAN: - cctx.gridEvents().record(new CacheQueryReadEvent<>( - cctx.localNode(), - "Scan query entry read.", - EVT_CACHE_QUERY_OBJECT_READ, - CacheQueryType.SCAN.name(), - cctx.name(), - null, - null, - qry.scanFilter(), - null, - null, - qry.subjectId(), - taskName, - key0, - val0, - null, - null)); + metrics.onRead(true); - break; + metrics.addGetTimeNanos(System.nanoTime() - start); } - } - if (rdc != null || trans != null) { - if (key0 == null) + K key0 = null; + V val0 = null; + + if (readEvt && cctx.gridEvents().hasListener(EVT_CACHE_QUERY_OBJECT_READ)) { key0 = (K)CacheObjectUtils.unwrapBinaryIfNeeded(objCtx, key, qry.keepBinary(), false); - if (val0 == null) val0 = (V)CacheObjectUtils.unwrapBinaryIfNeeded(objCtx, val, qry.keepBinary(), false); - Cache.Entry entry = new CacheEntryImpl(key0, val0); + switch (type) { + case SQL: + cctx.gridEvents().record(new CacheQueryReadEvent<>( + cctx.localNode(), + "SQL query entry read.", + EVT_CACHE_QUERY_OBJECT_READ, + CacheQueryType.SQL.name(), + cctx.name(), + qry.queryClassName(), + qry.clause(), + null, + null, + qryInfo.arguments(), + qry.subjectId(), + taskName, + key0, + val0, + null, + null)); + + break; + + case TEXT: + cctx.gridEvents().record(new CacheQueryReadEvent<>( + cctx.localNode(), + "Full text query entry read.", + EVT_CACHE_QUERY_OBJECT_READ, + CacheQueryType.FULL_TEXT.name(), + cctx.name(), + qry.queryClassName(), + qry.clause(), + null, + null, + null, + qry.subjectId(), + taskName, + key0, + val0, + null, + null)); + + break; + } + } - // Reduce. if (rdc != null) { + if (key0 == null) + key0 = (K)CacheObjectUtils.unwrapBinaryIfNeeded(objCtx, key, qry.keepBinary(), false); + if (val0 == null) + val0 = (V)CacheObjectUtils.unwrapBinaryIfNeeded(objCtx, val, qry.keepBinary(), false); + + Cache.Entry entry = new CacheEntryImpl(key0, val0); + + // Reduce. if (!rdc.collect(entry) || !iter.hasNext()) { onPageReady(loc, qryInfo, Collections.singletonList(rdc.reduce()), true, null); @@ -1312,12 +1285,9 @@ protected void runQuery(GridCacheQueryInfo qryInfo) { 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)); } - else - data.add(!loc ? new GridCacheQueryResponseEntry<>(key, val) : F.t(key, val)); if (!loc) { if (++cnt == pageSize || !iter.hasNext()) { @@ -1440,7 +1410,11 @@ protected GridCloseableIterator scanQueryLocal(final GridCacheQueryAdapter qry, taskName)); } - GridCloseableIterator it = scanIterator(qry, true); + IgniteClosure transformer = qry.transform(); + + injectResources(transformer); + + GridCloseableIterator it = scanIterator(qry, transformer, true); updateStatistics = false; @@ -1521,7 +1495,7 @@ protected GridCloseableIterator scanQueryLocal(final GridCacheQueryAdapter qry, if (exec) { try { - fut.onDone(executeQuery(qryInfo.query(), qryInfo.arguments(), false, + fut.onDone(executeQuery(qryInfo.query(), qryInfo.arguments(), qryInfo.transformer(), false, qryInfo.query().subjectId(), taskName, recipient(qryInfo.senderId(), qryInfo.requestId()))); } catch (Throwable e) { @@ -2762,6 +2736,7 @@ public CacheQuery createScanQuery(@Nullable IgniteBiPredicate filte * Creates user's predicate based scan query. * * @param filter Scan filter. + * @param trans Transformer. * @param part Partition. * @param keepBinary Keep binary flag. * @return Created query. @@ -2929,12 +2904,19 @@ private static final class ScanQueryIterator extends GridCloseableIterator /** */ private IgniteCacheExpiryPolicy expiryPlc; + /** */ + private final boolean locNode; + + /** */ + private final boolean incBackups; + /** * @param it Iterator. * @param qry Query. * @param topVer Topology version. * @param locPart Local partition. * @param scanFilter Scan filter. + * @param transformer Transformer. * @param locNode Local node flag. * @param cctx Cache context. * @param log Logger. @@ -2945,6 +2927,7 @@ private static final class ScanQueryIterator extends GridCloseableIterator AffinityTopologyVersion topVer, GridDhtLocalPartition locPart, IgniteBiPredicate scanFilter, + IgniteClosure transformer, boolean locNode, GridCacheContext cctx, IgniteLogger log) { @@ -2954,10 +2937,14 @@ private static final class ScanQueryIterator extends GridCloseableIterator this.scanFilter = scanFilter; this.cctx = cctx; this.log = log; + this.locNode = locNode; + + incBackups = qry.includeBackups(); - statsEnabled = locNode && cctx.statisticsEnabled(); + statsEnabled = cctx.statisticsEnabled(); - readEvt = locNode && cctx.gridEvents().isRecordable(EVT_CACHE_QUERY_OBJECT_READ); + readEvt = cctx.events().isRecordable(EVT_CACHE_QUERY_OBJECT_READ) && + cctx.gridEvents().hasListener(EVT_CACHE_QUERY_OBJECT_READ); if(readEvt){ taskName = cctx.kernalContext().task().resolveTaskName(qry.taskHash()); @@ -2969,8 +2956,8 @@ private static final class ScanQueryIterator extends GridCloseableIterator } // keep binary for remote scans if possible - keepBinary = (!locNode && scanFilter == null) || qry.keepBinary(); - transform = qry.transform(); + keepBinary = (!locNode && scanFilter == null && transformer == null && !readEvt) || qry.keepBinary(); + transform = transformer; dht = cctx.isLocal() ? null : (cctx.isNear() ? cctx.near().dht() : cctx.dht()); cache = dht != null ? dht : cctx.cache(); objCtx = cctx.cacheObjectContext(); @@ -3024,7 +3011,7 @@ private static final class ScanQueryIterator extends GridCloseableIterator private void advance() { long start = statsEnabled ? System.nanoTime() : 0L; - Object next = null; + Object next0 = null; while (it.hasNext()) { CacheDataRow row = it.next(); @@ -3066,6 +3053,31 @@ private void advance() { else val = row.value(); + // Filter backups for SCAN queries, if it isn't partition scan. + // Other types are filtered in indexing manager. + if (!cctx.isReplicated() && /*qry.partition()*/this.locPart == null && + cctx.config().getCacheMode() != LOCAL && !incBackups && + !cctx.affinity().primaryByKey(cctx.localNode(), key, topVer)) { + if (log.isDebugEnabled()) + log.debug("Ignoring backup element [row=" + row + + ", cacheMode=" + cctx.config().getCacheMode() + ", incBackups=" + incBackups + + ", primary=" + cctx.affinity().primaryByKey(cctx.localNode(), key, topVer) + ']'); + + continue; + } + + if (log.isDebugEnabled()) { + ClusterNode primaryNode = cctx.affinity().primaryByKey(key, + cctx.affinity().affinityTopologyVersion()); + + 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) { K key0 = (K)CacheObjectUtils.unwrapBinaryIfNeeded(objCtx, key, keepBinary, false); V val0 = (V)CacheObjectUtils.unwrapBinaryIfNeeded(objCtx, val, keepBinary, false); @@ -3079,7 +3091,7 @@ private void advance() { } if (scanFilter == null || scanFilter.apply(key0, val0)) { - if (readEvt && cctx.gridEvents().hasListener(EVT_CACHE_QUERY_OBJECT_READ)) { + if (readEvt) { cctx.gridEvents().record(new CacheQueryReadEvent<>( cctx.localNode(), "Scan query entry read.", @@ -3099,15 +3111,18 @@ private void advance() { null)); } - next = transform == null ? new CacheQueryEntry<>(key0, val0) - : transform.apply(new CacheQueryEntry<>(key0, val0)); + if (transform != null) + next0 = transform.apply(new CacheQueryEntry<>(key0, val0)); + else + next0 = !locNode ? new GridCacheQueryResponseEntry<>(key0, val0): + new CacheQueryEntry<>(key0, val0); break; } } } - if ((this.next = next) == null && expiryPlc != null && dht != null) { + if ((this.next = next0) == null && expiryPlc != null && dht != null) { dht.sendTtlUpdateRequest(expiryPlc); expiryPlc = null; 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 c378b6f7f012e..269ae71462b3d 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 @@ -594,6 +594,85 @@ public void testUnsupported() throws Exception { } } + /** + * @throws Exception If failed. + */ + public void testPageSize() throws Exception { + IgniteCache cache = grid().createCache("test-cache"); + + int numEntries = 10_000; + int pageSize = 3; + + try { + for (int i = 0; i < numEntries; i++) + cache.put(i, new Value("str" + i, i)); + + IgniteClosure, Integer> transformer = + new IgniteClosure, Integer>() { + @Override public Integer apply(Cache.Entry e) { + return e.getValue().idx; + } + }; + + ScanQuery query = new ScanQuery<>(); + query.setPageSize(pageSize); + + List res = cache.query(query, transformer).getAll(); + + assertEquals(numEntries, res.size()); + + Collections.sort(res); + + for (int i = 0; i < numEntries; i++) + assertEquals(i, res.get(i).intValue()); + } + finally { + cache.destroy(); + } + } + + /** + * @throws Exception If failed. + */ + public void testLocalInjection() 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, Boolean> transformer = + new IgniteClosure, Boolean>() { + @IgniteInstanceResource + Ignite ignite; + + @Override public Boolean apply(Cache.Entry e) { + return ignite != null; + } + }; + + return ignite.cache("test-cache").query(new ScanQuery().setLocal(true), + transformer).getAll(); + } + }); + + List res = new ArrayList<>(F.flatCollections(lists)); + + assertEquals(50, res.size()); + + for (int i = 0; i < 50; i++) + assertEquals(Boolean.TRUE, res.get(i)); + } + finally { + cache.destroy(); + } + } + /** */ private static class Value { From d53504adb1d4276f79ede2401e2d1512fe0287ec Mon Sep 17 00:00:00 2001 From: devozerov Date: Thu, 22 Feb 2018 10:07:19 +0300 Subject: [PATCH 030/314] IGNITE-7253: JDBC thin streaming: fixed default local buffer size, improved error messages in case of unsupported SQL statements. --- .../ignite/internal/jdbc2/JdbcStreamingSelfTest.java | 2 +- .../internal/jdbc/thin/ConnectionPropertiesImpl.java | 9 ++++++--- .../ignite/internal/jdbc/thin/JdbcThinStatement.java | 2 +- .../internal/processors/query/h2/IgniteH2Indexing.java | 2 +- 4 files changed, 9 insertions(+), 6 deletions(-) diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcStreamingSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcStreamingSelfTest.java index ebb6bc9b340c8..4c7e07903c8de 100644 --- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcStreamingSelfTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcStreamingSelfTest.java @@ -287,7 +287,7 @@ protected void assertStatementForbidden(String sql) { return null; } - }, SQLException.class,"Only tuple based INSERT statements are supported in streaming mode"); + }, SQLException.class,"Streaming mode supports only INSERT commands without subqueries."); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java index bb0a39d6bf077..5539f7d06be1c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java @@ -23,6 +23,8 @@ import java.util.Properties; import javax.naming.RefAddr; import javax.naming.Reference; + +import org.apache.ignite.IgniteDataStreamer; import org.apache.ignite.configuration.ClientConnectorConfiguration; import org.apache.ignite.internal.processors.odbc.SqlStateCode; import org.apache.ignite.internal.util.typedef.F; @@ -114,15 +116,16 @@ public class ConnectionPropertiesImpl implements ConnectionProperties, Serializa "streamingPerNodeBufferSize", "Buffer size per cluster node during streaming", 0, false, 0, Integer.MAX_VALUE); - /** Buffer size per cluster node during streaming. */ + /** Server-size flush frequency during streaming. */ private LongProperty streamFlushFreq = new LongProperty( - "streamingFlushFrequency", "Buffer size per cluster node during streaming", + "streamingFlushFrequency", "Server-size flush frequency during streaming", 0, false, 0, Long.MAX_VALUE); /** Buffer size per cluster node during streaming. */ private IntegerProperty streamBatchSize = new IntegerProperty( "streamingBatchSize", "Batch size for streaming (number of commands to accumulate internally " + - "before actually sending over the wire)", 10, false, 1, Integer.MAX_VALUE); + "before actually sending over the wire)", IgniteDataStreamer.DFLT_PER_NODE_BUFFER_SIZE * 4, false, 1, + Integer.MAX_VALUE); /** Properties array. */ private final ConnectionProperty [] propsArray = { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java index 61d837c1d2206..b2fc5347ef8c3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java @@ -138,7 +138,7 @@ protected void execute0(JdbcStatementType stmtType, String sql, List arg if (conn.isStream()) { if (stmtType == JdbcStatementType.SELECT_STATEMENT_TYPE) - throw new SQLException("Only tuple based INSERT statements are supported in streaming mode.", + throw new SQLException("executeQuery() method is not allowed in streaming mode.", SqlStateCode.INTERNAL_ERROR, IgniteQueryErrorCode.UNSUPPORTED_OPERATION); 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 50b148ba4492c..49b30ac7a9817 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 @@ -2338,7 +2338,7 @@ private Collection tables(String cacheName) { /** {@inheritDoc} */ @Override public void checkStatementStreamable(PreparedStatement nativeStmt) { if (!GridSqlQueryParser.isStreamableInsertStatement(nativeStmt)) - throw new IgniteSQLException("Only tuple based INSERT statements are supported in streaming mode.", + throw new IgniteSQLException("Streaming mode supports only INSERT commands without subqueries.", IgniteQueryErrorCode.UNSUPPORTED_OPERATION); } From debc906a25d3e2d65db58e16307fae6f08460eeb Mon Sep 17 00:00:00 2001 From: devozerov Date: Tue, 27 Feb 2018 12:13:52 +0300 Subject: [PATCH 031/314] Revert "IGNITE-7253: JDBC thin streaming: fixed default local buffer size, improved error messages in case of unsupported SQL statements." This reverts commit d53504adb1d4276f79ede2401e2d1512fe0287ec. --- .../ignite/internal/jdbc2/JdbcStreamingSelfTest.java | 2 +- .../internal/jdbc/thin/ConnectionPropertiesImpl.java | 9 +++------ .../ignite/internal/jdbc/thin/JdbcThinStatement.java | 2 +- .../internal/processors/query/h2/IgniteH2Indexing.java | 2 +- 4 files changed, 6 insertions(+), 9 deletions(-) diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcStreamingSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcStreamingSelfTest.java index 4c7e07903c8de..ebb6bc9b340c8 100644 --- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcStreamingSelfTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcStreamingSelfTest.java @@ -287,7 +287,7 @@ protected void assertStatementForbidden(String sql) { return null; } - }, SQLException.class,"Streaming mode supports only INSERT commands without subqueries."); + }, SQLException.class,"Only tuple based INSERT statements are supported in streaming mode"); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java index 5539f7d06be1c..bb0a39d6bf077 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java @@ -23,8 +23,6 @@ import java.util.Properties; import javax.naming.RefAddr; import javax.naming.Reference; - -import org.apache.ignite.IgniteDataStreamer; import org.apache.ignite.configuration.ClientConnectorConfiguration; import org.apache.ignite.internal.processors.odbc.SqlStateCode; import org.apache.ignite.internal.util.typedef.F; @@ -116,16 +114,15 @@ public class ConnectionPropertiesImpl implements ConnectionProperties, Serializa "streamingPerNodeBufferSize", "Buffer size per cluster node during streaming", 0, false, 0, Integer.MAX_VALUE); - /** Server-size flush frequency during streaming. */ + /** Buffer size per cluster node during streaming. */ private LongProperty streamFlushFreq = new LongProperty( - "streamingFlushFrequency", "Server-size flush frequency during streaming", + "streamingFlushFrequency", "Buffer size per cluster node during streaming", 0, false, 0, Long.MAX_VALUE); /** Buffer size per cluster node during streaming. */ private IntegerProperty streamBatchSize = new IntegerProperty( "streamingBatchSize", "Batch size for streaming (number of commands to accumulate internally " + - "before actually sending over the wire)", IgniteDataStreamer.DFLT_PER_NODE_BUFFER_SIZE * 4, false, 1, - Integer.MAX_VALUE); + "before actually sending over the wire)", 10, false, 1, Integer.MAX_VALUE); /** Properties array. */ private final ConnectionProperty [] propsArray = { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java index b2fc5347ef8c3..61d837c1d2206 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java @@ -138,7 +138,7 @@ protected void execute0(JdbcStatementType stmtType, String sql, List arg if (conn.isStream()) { if (stmtType == JdbcStatementType.SELECT_STATEMENT_TYPE) - throw new SQLException("executeQuery() method is not allowed in streaming mode.", + throw new SQLException("Only tuple based INSERT statements are supported in streaming mode.", SqlStateCode.INTERNAL_ERROR, IgniteQueryErrorCode.UNSUPPORTED_OPERATION); 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 49b30ac7a9817..50b148ba4492c 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 @@ -2338,7 +2338,7 @@ private Collection tables(String cacheName) { /** {@inheritDoc} */ @Override public void checkStatementStreamable(PreparedStatement nativeStmt) { if (!GridSqlQueryParser.isStreamableInsertStatement(nativeStmt)) - throw new IgniteSQLException("Streaming mode supports only INSERT commands without subqueries.", + throw new IgniteSQLException("Only tuple based INSERT statements are supported in streaming mode.", IgniteQueryErrorCode.UNSUPPORTED_OPERATION); } From d8cf9e042c0e9afe65508c006d9d1c39779d78b9 Mon Sep 17 00:00:00 2001 From: devozerov Date: Tue, 27 Feb 2018 12:14:21 +0300 Subject: [PATCH 032/314] Revert "IGNITE-7253: Streaming mode for JDBC thin driver. This closes #3499." This reverts commit bc331f9de716c30d6f733e28821ab44da7ed0cf7. --- .../jdbc2/JdbcNoCacheStreamingSelfTest.java | 182 ----------- .../internal/jdbc2/JdbcStreamingSelfTest.java | 218 +++----------- .../jdbc/suite/IgniteJdbcDriverTestSuite.java | 9 +- .../jdbc/thin/JdbcThinStreamingSelfTest.java | 285 ------------------ .../jdbc/thin/ConnectionProperties.java | 66 ---- .../jdbc/thin/ConnectionPropertiesImpl.java | 128 +------- .../jdbc/thin/JdbcThinConnection.java | 69 ----- .../jdbc/thin/JdbcThinPreparedStatement.java | 18 +- .../internal/jdbc/thin/JdbcThinStatement.java | 55 +--- .../internal/jdbc/thin/JdbcThinTcpIo.java | 32 +- .../ignite/internal/jdbc2/JdbcConnection.java | 12 +- .../jdbc2/JdbcStreamedPreparedStatement.java | 2 +- .../cache/IgniteCacheProxyImpl.java | 4 +- .../odbc/jdbc/JdbcConnectionContext.java | 29 +- .../odbc/jdbc/JdbcRequestHandler.java | 82 ++--- .../processors/query/GridQueryIndexing.java | 32 +- .../processors/query/GridQueryProcessor.java | 87 +----- .../processors/query/SqlClientContext.java | 195 ------------ .../apache/ignite/internal/sql/SqlParser.java | 2 +- ...niteClientCacheInitializationFailTest.java | 15 +- .../ignite/testframework/GridTestUtils.java | 2 +- .../query/h2/DmlStatementsProcessor.java | 104 ++++--- .../processors/query/h2/IgniteH2Indexing.java | 121 ++------ .../query/h2/ddl/DdlStatementsProcessor.java | 22 +- .../processors/query/h2/dml/UpdatePlan.java | 2 +- .../query/h2/dml/UpdatePlanBuilder.java | 13 +- .../query/h2/sql/GridSqlQueryParser.java | 12 - .../GridCacheCrossCacheQuerySelfTest.java | 18 +- 28 files changed, 260 insertions(+), 1556 deletions(-) delete mode 100644 modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcNoCacheStreamingSelfTest.java delete mode 100644 modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinStreamingSelfTest.java delete mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/query/SqlClientContext.java diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcNoCacheStreamingSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcNoCacheStreamingSelfTest.java deleted file mode 100644 index 74c2820f06f5a..0000000000000 --- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcNoCacheStreamingSelfTest.java +++ /dev/null @@ -1,182 +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.jdbc2; - -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.PreparedStatement; -import java.util.Collections; -import java.util.Properties; -import org.apache.ignite.IgniteJdbcDriver; -import org.apache.ignite.IgniteLogger; -import org.apache.ignite.configuration.CacheConfiguration; -import org.apache.ignite.configuration.ConnectorConfiguration; -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.vm.TcpDiscoveryVmIpFinder; -import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; - -import static org.apache.ignite.IgniteJdbcDriver.CFG_URL_PREFIX; -import static org.apache.ignite.cache.CacheMode.PARTITIONED; -import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; - -/** - * Data streaming test for thick driver and no explicit caches. - */ -public class JdbcNoCacheStreamingSelfTest extends GridCommonAbstractTest { - /** JDBC URL. */ - private static final String BASE_URL = CFG_URL_PREFIX + - "cache=default@modules/clients/src/test/config/jdbc-config.xml"; - - /** Connection. */ - protected Connection conn; - - /** */ - protected transient IgniteLogger log; - - /** {@inheritDoc} */ - @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { - return getConfiguration0(gridName); - } - - /** - * @param gridName Grid name. - * @return Grid configuration used for starting the grid. - * @throws Exception If failed. - */ - private IgniteConfiguration getConfiguration0(String gridName) throws Exception { - IgniteConfiguration cfg = super.getConfiguration(gridName); - - CacheConfiguration cache = defaultCacheConfiguration(); - - cache.setCacheMode(PARTITIONED); - cache.setBackups(1); - cache.setWriteSynchronizationMode(FULL_SYNC); - cache.setIndexedTypes( - Integer.class, Integer.class - ); - - cfg.setCacheConfiguration(cache); - cfg.setLocalHost("127.0.0.1"); - - TcpDiscoverySpi disco = new TcpDiscoverySpi(); - - TcpDiscoveryVmIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); - ipFinder.setAddresses(Collections.singleton("127.0.0.1:47500..47501")); - - disco.setIpFinder(ipFinder); - - cfg.setDiscoverySpi(disco); - - cfg.setConnectorConfiguration(new ConnectorConfiguration()); - - return cfg; - } - - /** {@inheritDoc} */ - @Override protected void beforeTestsStarted() throws Exception { - startGrids(2); - } - - /** {@inheritDoc} */ - @Override protected void afterTestsStopped() throws Exception { - stopAllGrids(); - } - - /** - * @param allowOverwrite Allow overwriting of existing keys. - * @return Connection to use for the test. - * @throws Exception if failed. - */ - protected Connection createConnection(boolean allowOverwrite) throws Exception { - Properties props = new Properties(); - - props.setProperty(IgniteJdbcDriver.PROP_STREAMING, "true"); - props.setProperty(IgniteJdbcDriver.PROP_STREAMING_FLUSH_FREQ, "500"); - - if (allowOverwrite) - props.setProperty(IgniteJdbcDriver.PROP_STREAMING_ALLOW_OVERWRITE, "true"); - - return DriverManager.getConnection(BASE_URL, props); - } - - /** {@inheritDoc} */ - @Override protected void afterTest() throws Exception { - U.closeQuiet(conn); - - ignite(0).cache(DEFAULT_CACHE_NAME).clear(); - - super.afterTest(); - } - - /** - * @throws Exception if failed. - */ - public void testStreamedInsert() throws Exception { - for (int i = 10; i <= 100; i += 10) - ignite(0).cache(DEFAULT_CACHE_NAME).put(i, i * 100); - - try (Connection conn = createConnection(false)) { - try (PreparedStatement stmt = conn.prepareStatement("insert into Integer(_key, _val) values (?, ?)")) { - for (int i = 1; i <= 100; i++) { - stmt.setInt(1, i); - stmt.setInt(2, i); - - stmt.executeUpdate(); - } - } - } - - U.sleep(500); - - // Now let's check it's all there. - for (int i = 1; i <= 100; i++) { - if (i % 10 != 0) - assertEquals(i, grid(0).cache(DEFAULT_CACHE_NAME).get(i)); - else // All that divides by 10 evenly should point to numbers 100 times greater - see above - assertEquals(i * 100, grid(0).cache(DEFAULT_CACHE_NAME).get(i)); - } - } - - /** - * @throws Exception if failed. - */ - public void testStreamedInsertWithOverwritesAllowed() throws Exception { - for (int i = 10; i <= 100; i += 10) - ignite(0).cache(DEFAULT_CACHE_NAME).put(i, i * 100); - - try (Connection conn = createConnection(true)) { - try (PreparedStatement stmt = conn.prepareStatement("insert into Integer(_key, _val) values (?, ?)")) { - for (int i = 1; i <= 100; i++) { - stmt.setInt(1, i); - stmt.setInt(2, i); - - stmt.executeUpdate(); - } - } - } - - U.sleep(500); - - // Now let's check it's all there. - // i should point to i at all times as we've turned overwrites on above. - for (int i = 1; i <= 100; i++) - assertEquals(i, grid(0).cache(DEFAULT_CACHE_NAME).get(i)); - } -} diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcStreamingSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcStreamingSelfTest.java index ebb6bc9b340c8..5418ca0dd0551 100644 --- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcStreamingSelfTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcStreamingSelfTest.java @@ -20,24 +20,16 @@ import java.sql.Connection; import java.sql.DriverManager; import java.sql.PreparedStatement; -import java.sql.SQLException; -import java.sql.Statement; import java.util.Collections; import java.util.Properties; -import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteJdbcDriver; import org.apache.ignite.IgniteLogger; -import org.apache.ignite.binary.BinaryObject; -import org.apache.ignite.binary.BinaryObjectBuilder; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.ConnectorConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; -import org.apache.ignite.internal.processors.query.QueryUtils; import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.lang.IgniteCallable; 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.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import static org.apache.ignite.IgniteJdbcDriver.CFG_URL_PREFIX; @@ -49,12 +41,10 @@ */ public class JdbcStreamingSelfTest extends GridCommonAbstractTest { /** JDBC URL. */ - private static final String BASE_URL = CFG_URL_PREFIX + - "cache=default@modules/clients/src/test/config/jdbc-config.xml"; + private static final String BASE_URL = CFG_URL_PREFIX + "cache=default@modules/clients/src/test/config/jdbc-config.xml"; - /** Streaming URL. */ - private static final String STREAMING_URL = CFG_URL_PREFIX + - "cache=person@modules/clients/src/test/config/jdbc-config.xml"; + /** Connection. */ + protected Connection conn; /** */ protected transient IgniteLogger log; @@ -100,18 +90,7 @@ private IgniteConfiguration getConfiguration0(String gridName) throws Exception /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { - super.beforeTestsStarted(); - startGrids(2); - - try (Connection c = createOrdinaryConnection()) { - try (Statement s = c.createStatement()) { - s.execute("CREATE TABLE PUBLIC.Person(\"id\" int primary key, \"name\" varchar) WITH " + - "\"cache_name=person,value_type=Person\""); - } - } - - U.sleep(1000); } /** {@inheritDoc} */ @@ -119,52 +98,28 @@ private IgniteConfiguration getConfiguration0(String gridName) throws Exception stopAllGrids(); } - /** - * @return Connection without streaming initially turned on. - * @throws SQLException if failed. - */ - protected Connection createOrdinaryConnection() throws SQLException { - Connection res = DriverManager.getConnection(BASE_URL, new Properties()); - - res.setSchema(QueryUtils.DFLT_SCHEMA); - - return res; - } - - /** - * @param allowOverwrite Allow overwriting of existing keys. - * @return Connection to use for the test. - * @throws Exception if failed. - */ - protected Connection createStreamedConnection(boolean allowOverwrite) throws Exception { - return createStreamedConnection(allowOverwrite, 500); - } - /** * @param allowOverwrite Allow overwriting of existing keys. - * @param flushTimeout Stream flush timeout. * @return Connection to use for the test. * @throws Exception if failed. */ - protected Connection createStreamedConnection(boolean allowOverwrite, long flushTimeout) throws Exception { + private Connection createConnection(boolean allowOverwrite) throws Exception { Properties props = new Properties(); props.setProperty(IgniteJdbcDriver.PROP_STREAMING, "true"); - props.setProperty(IgniteJdbcDriver.PROP_STREAMING_FLUSH_FREQ, String.valueOf(flushTimeout)); + props.setProperty(IgniteJdbcDriver.PROP_STREAMING_FLUSH_FREQ, "500"); if (allowOverwrite) props.setProperty(IgniteJdbcDriver.PROP_STREAMING_ALLOW_OVERWRITE, "true"); - Connection res = DriverManager.getConnection(STREAMING_URL, props); - - res.setSchema(QueryUtils.DFLT_SCHEMA); - - return res; + return DriverManager.getConnection(BASE_URL, props); } /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { - cache().clear(); + U.closeQuiet(conn); + + ignite(0).cache(DEFAULT_CACHE_NAME).clear(); super.afterTest(); } @@ -173,59 +128,30 @@ protected Connection createStreamedConnection(boolean allowOverwrite, long flush * @throws Exception if failed. */ public void testStreamedInsert() throws Exception { + conn = createConnection(false); + for (int i = 10; i <= 100; i += 10) - put(i, nameForId(i * 100)); - - try (Connection conn = createStreamedConnection(false)) { - try (PreparedStatement stmt = conn.prepareStatement("insert into PUBLIC.Person(\"id\", \"name\") " + - "values (?, ?)")) { - for (int i = 1; i <= 100; i++) { - stmt.setInt(1, i); - stmt.setString(2, nameForId(i)); - - stmt.executeUpdate(); - } - } - } + ignite(0).cache(DEFAULT_CACHE_NAME).put(i, i * 100); - U.sleep(500); + PreparedStatement stmt = conn.prepareStatement("insert into Integer(_key, _val) values (?, ?)"); - // Now let's check it's all there. for (int i = 1; i <= 100; i++) { - if (i % 10 != 0) - assertEquals(nameForId(i), nameForIdInCache(i)); - else // All that divides by 10 evenly should point to numbers 100 times greater - see above - assertEquals(nameForId(i * 100), nameForIdInCache(i)); - } - } + stmt.setInt(1, i); + stmt.setInt(2, i); - /** - * @throws Exception if failed. - */ - public void testStreamedInsertWithoutColumnsList() throws Exception { - for (int i = 10; i <= 100; i += 10) - put(i, nameForId(i * 100)); - - try (Connection conn = createStreamedConnection(false)) { - try (PreparedStatement stmt = conn.prepareStatement("insert into PUBLIC.Person(\"id\", \"name\") " + - "values (?, ?)")) { - for (int i = 1; i <= 100; i++) { - stmt.setInt(1, i); - stmt.setString(2, nameForId(i)); - - stmt.executeUpdate(); - } - } + stmt.executeUpdate(); } - U.sleep(500); + // Closing connection makes it wait for streamer close + // and thus for data load completion as well + conn.close(); // Now let's check it's all there. for (int i = 1; i <= 100; i++) { if (i % 10 != 0) - assertEquals(nameForId(i), nameForIdInCache(i)); + assertEquals(i, grid(0).cache(DEFAULT_CACHE_NAME).get(i)); else // All that divides by 10 evenly should point to numbers 100 times greater - see above - assertEquals(nameForId(i * 100), nameForIdInCache(i)); + assertEquals(i * 100, grid(0).cache(DEFAULT_CACHE_NAME).get(i)); } } @@ -233,99 +159,27 @@ public void testStreamedInsertWithoutColumnsList() throws Exception { * @throws Exception if failed. */ public void testStreamedInsertWithOverwritesAllowed() throws Exception { + conn = createConnection(true); + for (int i = 10; i <= 100; i += 10) - put(i, nameForId(i * 100)); - - try (Connection conn = createStreamedConnection(true)) { - try (PreparedStatement stmt = conn.prepareStatement("insert into PUBLIC.Person(\"id\", \"name\") " + - "values (?, ?)")) { - for (int i = 1; i <= 100; i++) { - stmt.setInt(1, i); - stmt.setString(2, nameForId(i)); - - stmt.executeUpdate(); - } - } + ignite(0).cache(DEFAULT_CACHE_NAME).put(i, i * 100); + + PreparedStatement stmt = conn.prepareStatement("insert into Integer(_key, _val) values (?, ?)"); + + for (int i = 1; i <= 100; i++) { + stmt.setInt(1, i); + stmt.setInt(2, i); + + stmt.executeUpdate(); } - U.sleep(500); + // Closing connection makes it wait for streamer close + // and thus for data load completion as well + conn.close(); // Now let's check it's all there. // i should point to i at all times as we've turned overwrites on above. for (int i = 1; i <= 100; i++) - assertEquals(nameForId(i), nameForIdInCache(i)); - } - - /** */ - public void testOnlyInsertsAllowed() { - assertStatementForbidden("CREATE TABLE PUBLIC.X (x int primary key, y int)"); - - assertStatementForbidden("SELECT * from Person"); - - assertStatementForbidden("insert into PUBLIC.Person(\"id\", \"name\") " + - "(select \"id\" + 1, CONCAT(\"name\", '1') from Person)"); - - assertStatementForbidden("DELETE from Person"); - - assertStatementForbidden("UPDATE Person SET \"name\" = 'name0'"); - - assertStatementForbidden("alter table Person add column y int"); - } - - /** - * @param sql Statement to check. - */ - @SuppressWarnings("ThrowableNotThrown") - protected void assertStatementForbidden(String sql) { - GridTestUtils.assertThrows(null, new IgniteCallable() { - @Override public Object call() throws Exception { - try (Connection c = createStreamedConnection(false)) { - try (PreparedStatement s = c.prepareStatement(sql)) { - s.execute(); - } - } - - return null; - } - }, SQLException.class,"Only tuple based INSERT statements are supported in streaming mode"); - } - - /** - * @return Person cache. - */ - protected IgniteCache cache() { - return grid(0).cache("person"); - } - - /** - * @param id id of person to put. - * @param name name of person to put. - */ - protected void put(int id, String name) { - BinaryObjectBuilder bldr = grid(0).binary().builder("Person"); - - bldr.setField("name", name); - - cache().put(id, bldr.build()); - } - - /** - * @param id Person id. - * @return Default name for person w/given id. - */ - protected String nameForId(int id) { - return "Person" + id; - } - - /** - * @param id person id. - * @return Name for person with given id currently stored in cache. - */ - protected String nameForIdInCache(int id) { - Object o = cache().withKeepBinary().get(id); - - assertTrue(String.valueOf(o), o instanceof BinaryObject); - - return ((BinaryObject)o).field("name"); + assertEquals(i, grid(0).cache(DEFAULT_CACHE_NAME).get(i)); } } diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java index c380049db975b..656e218ecee09 100644 --- a/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java +++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java @@ -42,7 +42,6 @@ import org.apache.ignite.jdbc.thin.JdbcThinBulkLoadTransactionalPartitionedSelfTest; import org.apache.ignite.jdbc.thin.JdbcThinBulkLoadTransactionalReplicatedSelfTest; import org.apache.ignite.jdbc.thin.JdbcThinComplexDmlDdlSelfTest; -import org.apache.ignite.jdbc.thin.JdbcThinComplexDmlDdlSkipReducerOnUpdateSelfTest; import org.apache.ignite.jdbc.thin.JdbcThinComplexQuerySelfTest; import org.apache.ignite.jdbc.thin.JdbcThinConnectionSelfTest; import org.apache.ignite.jdbc.thin.JdbcThinDeleteStatementSelfTest; @@ -55,10 +54,8 @@ import org.apache.ignite.jdbc.thin.JdbcThinEmptyCacheSelfTest; import org.apache.ignite.jdbc.thin.JdbcThinErrorsSelfTest; import org.apache.ignite.jdbc.thin.JdbcThinInsertStatementSelfTest; -import org.apache.ignite.jdbc.thin.JdbcThinInsertStatementSkipReducerOnUpdateSelfTest; import org.apache.ignite.jdbc.thin.JdbcThinLocalQueriesSelfTest; import org.apache.ignite.jdbc.thin.JdbcThinMergeStatementSelfTest; -import org.apache.ignite.jdbc.thin.JdbcThinMergeStatementSkipReducerOnUpdateSelfTest; import org.apache.ignite.jdbc.thin.JdbcThinMetadataSelfTest; import org.apache.ignite.jdbc.thin.JdbcThinMissingLongArrayResultsTest; import org.apache.ignite.jdbc.thin.JdbcThinNoDefaultSchemaTest; @@ -67,8 +64,10 @@ import org.apache.ignite.jdbc.thin.JdbcThinSchemaCaseTest; import org.apache.ignite.jdbc.thin.JdbcThinSelectAfterAlterTable; import org.apache.ignite.jdbc.thin.JdbcThinStatementSelfTest; -import org.apache.ignite.jdbc.thin.JdbcThinStreamingSelfTest; import org.apache.ignite.jdbc.thin.JdbcThinUpdateStatementSelfTest; +import org.apache.ignite.jdbc.thin.JdbcThinComplexDmlDdlSkipReducerOnUpdateSelfTest; +import org.apache.ignite.jdbc.thin.JdbcThinInsertStatementSkipReducerOnUpdateSelfTest; +import org.apache.ignite.jdbc.thin.JdbcThinMergeStatementSkipReducerOnUpdateSelfTest; import org.apache.ignite.jdbc.thin.JdbcThinUpdateStatementSkipReducerOnUpdateSelfTest; import org.apache.ignite.jdbc.thin.JdbcThinWalModeChangeSelfTest; @@ -120,11 +119,9 @@ public static TestSuite suite() throws Exception { suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcStatementBatchingSelfTest.class)); suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcErrorsSelfTest.class)); suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcStreamingToPublicCacheTest.class)); - suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcNoCacheStreamingSelfTest.class)); suite.addTest(new TestSuite(JdbcBlobTest.class)); suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcStreamingSelfTest.class)); - suite.addTest(new TestSuite(JdbcThinStreamingSelfTest.class)); // DDL tests. suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcDynamicIndexAtomicPartitionedNearSelfTest.class)); diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinStreamingSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinStreamingSelfTest.java deleted file mode 100644 index 9eba4da2a0a8a..0000000000000 --- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinStreamingSelfTest.java +++ /dev/null @@ -1,285 +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.jdbc.thin; - -import java.sql.Connection; -import java.sql.PreparedStatement; -import java.sql.SQLException; -import java.sql.Statement; -import java.util.Arrays; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Set; -import org.apache.ignite.IgniteDataStreamer; -import org.apache.ignite.cache.CachePeekMode; -import org.apache.ignite.internal.jdbc2.JdbcStreamingSelfTest; -import org.apache.ignite.internal.processors.query.SqlClientContext; -import org.apache.ignite.internal.util.typedef.F; -import org.apache.ignite.internal.util.typedef.internal.U; - -/** - * Tests for streaming via thin driver. - */ -public class JdbcThinStreamingSelfTest extends JdbcStreamingSelfTest { - /** */ - private int batchSize = 17; - - /** {@inheritDoc} */ - @Override protected void beforeTestsStarted() throws Exception { - super.beforeTestsStarted(); - - batchSize = 17; - } - - /** {@inheritDoc} */ - @Override protected void afterTest() throws Exception { - try (Connection c = createOrdinaryConnection()) { - execute(c, "DROP TABLE PUBLIC.T IF EXISTS"); - } - - super.afterTest(); - } - - /** {@inheritDoc} */ - @Override protected Connection createStreamedConnection(boolean allowOverwrite, long flushFreq) throws Exception { - return JdbcThinAbstractSelfTest.connect(grid(0), "streaming=true&streamingFlushFrequency=" - + flushFreq + "&" + "streamingAllowOverwrite=" + allowOverwrite + "&streamingPerNodeBufferSize=1000&" - + "streamingBatchSize=" + batchSize); - } - - /** {@inheritDoc} */ - @Override protected Connection createOrdinaryConnection() throws SQLException { - return JdbcThinAbstractSelfTest.connect(grid(0), null); - } - - /** - * @throws Exception if failed. - */ - public void testStreamedBatchedInsert() throws Exception { - for (int i = 10; i <= 100; i += 10) - put(i, nameForId(i * 100)); - - try (Connection conn = createStreamedConnection(false)) { - assertStreamingOn(); - - try (PreparedStatement stmt = conn.prepareStatement("insert into Person(\"id\", \"name\") values (?, ?), " + - "(?, ?)")) { - for (int i = 1; i <= 100; i+=2) { - stmt.setInt(1, i); - stmt.setString(2, nameForId(i)); - stmt.setInt(3, i + 1); - stmt.setString(4, nameForId(i + 1)); - - stmt.addBatch(); - } - - stmt.executeBatch(); - } - } - - U.sleep(500); - - // Now let's check it's all there. - for (int i = 1; i <= 100; i++) { - if (i % 10 != 0) - assertEquals(nameForId(i), nameForIdInCache(i)); - else // All that divides by 10 evenly should point to numbers 100 times greater - see above - assertEquals(nameForId(i * 100), nameForIdInCache(i)); - } - } - - /** - * @throws SQLException if failed. - */ - public void testSimultaneousStreaming() throws Exception { - try (Connection anotherConn = createOrdinaryConnection()) { - execute(anotherConn, "CREATE TABLE PUBLIC.T(x int primary key, y int) WITH " + - "\"cache_name=T,wrap_value=false\""); - } - - // Timeout to let connection close be handled on server side. - U.sleep(500); - - try (Connection conn = createStreamedConnection(false, 10000)) { - assertStreamingOn(); - - PreparedStatement firstStmt = conn.prepareStatement("insert into Person(\"id\", \"name\") values (?, ?)"); - - PreparedStatement secondStmt = conn.prepareStatement("insert into PUBLIC.T(x, y) values (?, ?)"); - - try { - for (int i = 1; i <= 10; i++) { - firstStmt.setInt(1, i); - firstStmt.setString(2, nameForId(i)); - - firstStmt.executeUpdate(); - } - - for (int i = 51; i <= 67; i++) { - secondStmt.setInt(1, i); - secondStmt.setInt(2, i); - - secondStmt.executeUpdate(); - } - - for (int i = 11; i <= 50; i++) { - firstStmt.setInt(1, i); - firstStmt.setString(2, nameForId(i)); - - firstStmt.executeUpdate(); - } - - for (int i = 68; i <= 100; i++) { - secondStmt.setInt(1, i); - secondStmt.setInt(2, i); - - secondStmt.executeUpdate(); - } - - assertCacheEmpty(); - - SqlClientContext cliCtx = sqlClientContext(); - - HashMap> streamers = U.field(cliCtx, "streamers"); - - assertEquals(2, streamers.size()); - - assertEqualsCollections(new HashSet<>(Arrays.asList("person", "T")), streamers.keySet()); - } - finally { - U.closeQuiet(firstStmt); - - U.closeQuiet(secondStmt); - } - } - - // Let's wait a little so that all data arrives to destination - we can't intercept streamers' flush - // on connection close in any way. - U.sleep(1000); - - // Now let's check it's all there. - for (int i = 1; i <= 50; i++) - assertEquals(nameForId(i), nameForIdInCache(i)); - - for (int i = 51; i <= 100; i++) - assertEquals(i, grid(0).cache("T").get(i)); - } - - /** - * - */ - public void testStreamingWithMixedStatementTypes() throws Exception { - String prepStmtStr = "insert into Person(\"id\", \"name\") values (?, ?)"; - - String stmtStr = "insert into Person(\"id\", \"name\") values (%d, '%s')"; - - try (Connection conn = createStreamedConnection(false, 10000)) { - assertStreamingOn(); - - PreparedStatement firstStmt = conn.prepareStatement(prepStmtStr); - - Statement secondStmt = conn.createStatement(); - - try { - for (int i = 1; i <= 100; i++) { - boolean usePrep = Math.random() > 0.5; - - boolean useBatch = Math.random() > 0.5; - - if (usePrep) { - firstStmt.setInt(1, i); - firstStmt.setString(2, nameForId(i)); - - if (useBatch) - firstStmt.addBatch(); - else - firstStmt.execute(); - } - else { - String sql = String.format(stmtStr, i, nameForId(i)); - - if (useBatch) - secondStmt.addBatch(sql); - else - secondStmt.execute(sql); - } - } - } - finally { - U.closeQuiet(firstStmt); - - U.closeQuiet(secondStmt); - } - } - - // Let's wait a little so that all data arrives to destination - we can't intercept streamers' flush - // on connection close in any way. - U.sleep(1000); - - // Now let's check it's all there. - for (int i = 1; i <= 100; i++) - assertEquals(nameForId(i), nameForIdInCache(i)); - } - - /** - * Check that there's nothing in cache. - */ - private void assertCacheEmpty() { - assertEquals(0, grid(0).cache(DEFAULT_CACHE_NAME).size(CachePeekMode.ALL)); - } - - /** - * @param conn Connection. - * @param sql Statement. - * @throws SQLException if failed. - */ - private static void execute(Connection conn, String sql) throws SQLException { - try (Statement s = conn.createStatement()) { - s.execute(sql); - } - } - - /** - * @return Active SQL client context. - */ - private SqlClientContext sqlClientContext() { - Set ctxs = U.field(grid(0).context().query(), "cliCtxs"); - - assertFalse(F.isEmpty(ctxs)); - - assertEquals(1, ctxs.size()); - - return ctxs.iterator().next(); - } - - /** - * Check that streaming state on target node is as expected. - */ - private void assertStreamingOn() { - SqlClientContext cliCtx = sqlClientContext(); - - assertTrue(cliCtx.isStream()); - } - - /** {@inheritDoc} */ - @Override protected void assertStatementForbidden(String sql) { - batchSize = 1; - - super.assertStatementForbidden(sql); - } -} \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionProperties.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionProperties.java index 458facad23086..d79348415f4cc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionProperties.java @@ -145,70 +145,4 @@ public interface ConnectionProperties { * @param skipReducerOnUpdate Skip reducer on update flag. */ public void setSkipReducerOnUpdate(boolean skipReducerOnUpdate); - - /** - * @return Streamed connection flag. - */ - public boolean isStream(); - - /** - * @param stream Streamed connection flag. - */ - public void setStream(boolean stream); - - /** - * @return Allow overwrites during streaming connection flag. - */ - public boolean isStreamAllowOverwrite(); - - /** - * @param streamAllowOverwrite Allow overwrites during streaming connection flag. - */ - public void setStreamAllowOverwrite(boolean streamAllowOverwrite); - - /** - * @return Number of parallel operations per node during streaming connection param. - */ - public int getStreamParallelOperations(); - - /** - * @param streamParallelOperations Number of parallel operations per node during streaming connection param. - * @throws SQLException if value check failed. - */ - public void setStreamParallelOperations(int streamParallelOperations) throws SQLException; - - /** - * @return Buffer size during streaming connection param. - */ - public int getStreamBufferSize(); - - /** - * @param streamBufSize Buffer size during streaming connection param. - * @throws SQLException if value check failed. - */ - public void setStreamBufferSize(int streamBufSize) throws SQLException; - - /** - * @return Flush timeout during streaming connection param. - */ - public long getStreamFlushFrequency(); - - /** - * @param streamFlushFreq Flush timeout during streaming connection param. - * @throws SQLException if value check failed. - */ - public void setStreamFlushFrequency(long streamFlushFreq) throws SQLException; - - /** - * @return Batch size for streaming (number of commands to accumulate internally before actually - * sending over the wire). - */ - public int getStreamBatchSize(); - - /** - * @param streamBatchSize Batch size for streaming (number of commands to accumulate internally before actually - * sending over the wire). - * @throws SQLException if value check failed. - */ - public void setStreamBatchSize(int streamBatchSize) throws SQLException; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java index bb0a39d6bf077..86ba2fa832b1d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java @@ -96,40 +96,11 @@ public class ConnectionPropertiesImpl implements ConnectionProperties, Serializa private BooleanProperty skipReducerOnUpdate = new BooleanProperty( "skipReducerOnUpdate", "Enable execution update queries on ignite server nodes", false, false); - /** Turn on streaming mode on this connection. */ - private BooleanProperty stream = new BooleanProperty( - "streaming", "Turn on streaming mode on this connection", false, false); - - /** Turn on overwrite during streaming on this connection. */ - private BooleanProperty streamAllowOverwrite = new BooleanProperty( - "streamingAllowOverwrite", "Turn on overwrite during streaming on this connection", false, false); - - /** Number of parallel operations per cluster node during streaming. */ - private IntegerProperty streamParOps = new IntegerProperty( - "streamingPerNodeParallelOperations", "Number of parallel operations per cluster node during streaming", - 0, false, 0, Integer.MAX_VALUE); - - /** Buffer size per cluster node during streaming. */ - private IntegerProperty streamBufSize = new IntegerProperty( - "streamingPerNodeBufferSize", "Buffer size per cluster node during streaming", - 0, false, 0, Integer.MAX_VALUE); - - /** Buffer size per cluster node during streaming. */ - private LongProperty streamFlushFreq = new LongProperty( - "streamingFlushFrequency", "Buffer size per cluster node during streaming", - 0, false, 0, Long.MAX_VALUE); - - /** Buffer size per cluster node during streaming. */ - private IntegerProperty streamBatchSize = new IntegerProperty( - "streamingBatchSize", "Batch size for streaming (number of commands to accumulate internally " + - "before actually sending over the wire)", 10, false, 1, Integer.MAX_VALUE); - /** Properties array. */ private final ConnectionProperty [] propsArray = { host, port, distributedJoins, enforceJoinOrder, collocated, replicatedOnly, autoCloseServerCursor, - tcpNoDelay, lazy, socketSendBuffer, socketReceiveBuffer, skipReducerOnUpdate, - stream, streamAllowOverwrite, streamParOps, streamBufSize, streamFlushFreq, streamBatchSize + tcpNoDelay, lazy, socketSendBuffer, socketReceiveBuffer, skipReducerOnUpdate }; /** {@inheritDoc} */ @@ -252,66 +223,6 @@ public class ConnectionPropertiesImpl implements ConnectionProperties, Serializa skipReducerOnUpdate.setValue(val); } - /** {@inheritDoc} */ - @Override public boolean isStream() { - return stream.value(); - } - - /** {@inheritDoc} */ - @Override public void setStream(boolean val) { - stream.setValue(val); - } - - /** {@inheritDoc} */ - @Override public boolean isStreamAllowOverwrite() { - return streamAllowOverwrite.value(); - } - - /** {@inheritDoc} */ - @Override public void setStreamAllowOverwrite(boolean val) { - streamAllowOverwrite.setValue(val); - } - - /** {@inheritDoc} */ - @Override public int getStreamParallelOperations() { - return streamParOps.value(); - } - - /** {@inheritDoc} */ - @Override public void setStreamParallelOperations(int val) throws SQLException { - streamParOps.setValue(val); - } - - /** {@inheritDoc} */ - @Override public int getStreamBufferSize() { - return streamBufSize.value(); - } - - /** {@inheritDoc} */ - @Override public void setStreamBufferSize(int val) throws SQLException { - streamBufSize.setValue(val); - } - - /** {@inheritDoc} */ - @Override public long getStreamFlushFrequency() { - return streamFlushFreq.value(); - } - - /** {@inheritDoc} */ - @Override public void setStreamFlushFrequency(long val) throws SQLException { - streamFlushFreq.setValue(val); - } - - /** {@inheritDoc} */ - @Override public int getStreamBatchSize() { - return streamBatchSize.value(); - } - - /** {@inheritDoc} */ - @Override public void setStreamBatchSize(int val) throws SQLException { - streamBatchSize.setValue(val); - } - /** * @param props Environment properties. * @throws SQLException On error. @@ -319,7 +230,7 @@ public class ConnectionPropertiesImpl implements ConnectionProperties, Serializa void init(Properties props) throws SQLException { Properties props0 = (Properties)props.clone(); - for (ConnectionProperty aPropsArray : this.propsArray) + for (ConnectionProperty aPropsArray : propsArray) aPropsArray.init(props0); } @@ -602,8 +513,7 @@ private abstract static class NumberProperty extends ConnectionProperty { else { try { setValue(parse(str)); - } - catch (NumberFormatException e) { + } catch (NumberFormatException e) { throw new SQLException("Failed to parse int property [name=" + name + ", value=" + str + ']', SqlStateCode.CLIENT_CONNECTION_FAILED); } @@ -675,38 +585,6 @@ int value() { } } - /** - * - */ - private static class LongProperty extends NumberProperty { - /** */ - private static final long serialVersionUID = 0L; - - /** - * @param name Name. - * @param desc Description. - * @param dfltVal Default value. - * @param required {@code true} if the property is required. - * @param min Lower bound of allowed range. - * @param max Upper bound of allowed range. - */ - LongProperty(String name, String desc, Number dfltVal, boolean required, long min, long max) { - super(name, desc, dfltVal, required, min, max); - } - - /** {@inheritDoc} */ - @Override protected Number parse(String str) throws NumberFormatException { - return Long.parseLong(str); - } - - /** - * @return Property value. - */ - long value() { - return val.longValue(); - } - } - /** * */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java index d267070718f28..999c793d55ccf 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java @@ -18,7 +18,6 @@ package org.apache.ignite.internal.jdbc.thin; import java.sql.Array; -import java.sql.BatchUpdateException; import java.sql.Blob; import java.sql.CallableStatement; import java.sql.Clob; @@ -34,19 +33,13 @@ import java.sql.Savepoint; import java.sql.Statement; import java.sql.Struct; -import java.util.ArrayList; -import java.util.List; import java.util.Map; import java.util.Properties; import java.util.concurrent.Executor; -import java.util.logging.Level; import java.util.logging.Logger; import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode; import org.apache.ignite.internal.processors.odbc.ClientListenerResponse; import org.apache.ignite.internal.processors.odbc.SqlStateCode; -import org.apache.ignite.internal.processors.odbc.jdbc.JdbcBatchExecuteRequest; -import org.apache.ignite.internal.processors.odbc.jdbc.JdbcBatchExecuteResult; -import org.apache.ignite.internal.processors.odbc.jdbc.JdbcQuery; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcResponse; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcResult; @@ -101,12 +94,6 @@ public class JdbcThinConnection implements Connection { /** Connection properties. */ private ConnectionProperties connProps; - /** Batch for streaming. */ - private List streamBatch; - - /** Last added query to recognize batches. */ - private String lastStreamQry; - /** * Creates new connection. * @@ -143,53 +130,6 @@ public JdbcThinConnection(String url, String schema, Properties props) throws SQ } } - /** - * @return Whether this connection is streamed or not. - */ - public boolean isStream() { - return connProps.isStream(); - } - - /** - * Add another query for batched execution. - * @param sql Query. - * @param args Arguments. - */ - synchronized void addBatch(String sql, List args) throws SQLException { - boolean newQry = (args == null || !F.eq(lastStreamQry, sql)); - - // Providing null as SQL here allows for recognizing subbatches on server and handling them more efficiently. - JdbcQuery q = new JdbcQuery(newQry ? sql : null, args != null ? args.toArray() : null); - - if (streamBatch == null) - streamBatch = new ArrayList<>(connProps.getStreamBatchSize()); - - streamBatch.add(q); - - // Null args means "addBatch(String)" was called on non-prepared Statement, - // we don't want to remember its query string. - lastStreamQry = (args != null ? sql : null); - - if (streamBatch.size() == connProps.getStreamBatchSize()) - executeBatch(); - } - - /** - * @throws SQLException if failed. - */ - private void executeBatch() throws SQLException { - JdbcBatchExecuteResult res = sendRequest(new JdbcBatchExecuteRequest(schema, streamBatch)); - - streamBatch = null; - - lastStreamQry = null; - - if (res.errorCode() != ClientListenerResponse.STATUS_SUCCESS) { - throw new BatchUpdateException(res.errorMessage(), IgniteQueryErrorCode.codeToSqlState(res.errorCode()), - res.errorCode(), res.updateCounts()); - } - } - /** {@inheritDoc} */ @Override public Statement createStatement() throws SQLException { return createStatement(TYPE_FORWARD_ONLY, CONCUR_READ_ONLY, HOLD_CURSORS_OVER_COMMIT); @@ -332,15 +272,6 @@ private void checkCursorOptions(int resSetType, int resSetConcurrency, if (isClosed()) return; - if (!F.isEmpty(streamBatch)) { - try { - executeBatch(); - } - catch (SQLException e) { - LOG.log(Level.WARNING, "Exception during batch send on streamed connection close", e); - } - } - closed = true; cliIo.close(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinPreparedStatement.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinPreparedStatement.java index b575167ea8965..23d3bbe37102b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinPreparedStatement.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinPreparedStatement.java @@ -39,8 +39,8 @@ import java.sql.Timestamp; import java.util.ArrayList; import java.util.Calendar; -import org.apache.ignite.internal.processors.odbc.SqlListenerUtils; import org.apache.ignite.internal.processors.odbc.SqlStateCode; +import org.apache.ignite.internal.processors.odbc.SqlListenerUtils; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMetaParamsRequest; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMetaParamsResult; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcQuery; @@ -262,19 +262,13 @@ private void executeWithArguments(JdbcStatementType stmtType) throws SQLExceptio @Override public void addBatch() throws SQLException { ensureNotClosed(); - batchSize++; - - if (conn.isStream()) - conn.addBatch(sql, args); - else { - if (batch == null) { - batch = new ArrayList<>(); + if (batch == null) { + batch = new ArrayList<>(); - batch.add(new JdbcQuery(sql, args.toArray(new Object[args.size()]))); - } - else - batch.add(new JdbcQuery(null, args.toArray(new Object[args.size()]))); + batch.add(new JdbcQuery(sql, args.toArray(new Object[args.size()]))); } + else + batch.add(new JdbcQuery(null, args.toArray(new Object[args.size()]))); args = null; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java index 61d837c1d2206..9c4180439adeb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java @@ -38,13 +38,13 @@ import org.apache.ignite.internal.processors.odbc.jdbc.JdbcBatchExecuteRequest; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcBatchExecuteResult; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcBulkLoadAckResult; -import org.apache.ignite.internal.processors.odbc.jdbc.JdbcBulkLoadBatchRequest; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcQuery; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcQueryExecuteMultipleStatementsResult; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcQueryExecuteRequest; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcQueryExecuteResult; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcResult; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcResultInfo; +import org.apache.ignite.internal.processors.odbc.jdbc.JdbcBulkLoadBatchRequest; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcStatementType; import static java.sql.ResultSet.CONCUR_READ_ONLY; @@ -79,9 +79,6 @@ public class JdbcThinStatement implements Statement { /** Result set holdability*/ private final int resHoldability; - /** Batch size to keep track of number of items to return as fake update counters for executeBatch. */ - protected int batchSize; - /** Batch. */ protected List batch; @@ -136,19 +133,6 @@ protected void execute0(JdbcStatementType stmtType, String sql, List arg if (sql == null || sql.isEmpty()) throw new SQLException("SQL query is empty."); - if (conn.isStream()) { - if (stmtType == JdbcStatementType.SELECT_STATEMENT_TYPE) - throw new SQLException("Only tuple based INSERT statements are supported in streaming mode.", - SqlStateCode.INTERNAL_ERROR, - IgniteQueryErrorCode.UNSUPPORTED_OPERATION); - - conn.addBatch(sql, args); - - resultSets = Collections.singletonList(resultSetForUpdate(0)); - - return; - } - JdbcResult res0 = conn.sendRequest(new JdbcQueryExecuteRequest(stmtType, schema, pageSize, maxRows, sql, args == null ? null : args.toArray(new Object[args.size()]))); @@ -174,8 +158,11 @@ else if (res0 instanceof JdbcQueryExecuteMultipleStatementsResult) { boolean firstRes = true; for(JdbcResultInfo rsInfo : resInfos) { - if (!rsInfo.isQuery()) - resultSets.add(resultSetForUpdate(rsInfo.updateCount())); + if (!rsInfo.isQuery()) { + resultSets.add(new JdbcThinResultSet(this, -1, pageSize, + true, Collections.>emptyList(), false, + conn.autoCloseServerCursor(), rsInfo.updateCount(), closeOnCompletion)); + } else { if (firstRes) { firstRes = false; @@ -198,16 +185,6 @@ else if (res0 instanceof JdbcQueryExecuteMultipleStatementsResult) { assert resultSets.size() > 0 : "At least one results set is expected"; } - /** - * @param cnt Update counter. - * @return Result set for given update counter. - */ - private JdbcThinResultSet resultSetForUpdate(long cnt) { - return new JdbcThinResultSet(this, -1, pageSize, - true, Collections.>emptyList(), false, - conn.autoCloseServerCursor(), cnt, closeOnCompletion); - } - /** * Sends a file to server in batches via multiple {@link JdbcBulkLoadBatchRequest}s. * @@ -492,14 +469,6 @@ private JdbcThinResultSet nextResultSet() throws SQLException { @Override public void addBatch(String sql) throws SQLException { ensureNotClosed(); - batchSize++; - - if (conn.isStream()) { - conn.addBatch(sql, null); - - return; - } - if (batch == null) batch = new ArrayList<>(); @@ -510,8 +479,6 @@ private JdbcThinResultSet nextResultSet() throws SQLException { @Override public void clearBatch() throws SQLException { ensureNotClosed(); - batchSize = 0; - batch = null; } @@ -521,14 +488,6 @@ private JdbcThinResultSet nextResultSet() throws SQLException { closeResults(); - if (conn.isStream()) { - int[] res = new int[batchSize]; - - batchSize = 0; - - return res; - } - if (batch == null || batch.isEmpty()) throw new SQLException("Batch is empty."); @@ -543,8 +502,6 @@ private JdbcThinResultSet nextResultSet() throws SQLException { return res.updateCounts(); } finally { - batchSize = 0; - batch = null; } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java index 79bdc5e9189c7..fec218e368494 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java @@ -20,9 +20,27 @@ import java.io.BufferedInputStream; import java.io.BufferedOutputStream; import java.io.IOException; +import java.io.InputStream; import java.net.InetSocketAddress; +import java.net.MalformedURLException; import java.net.Socket; +import java.net.URL; +import java.security.KeyManagementException; +import java.security.KeyStore; +import java.security.KeyStoreException; +import java.security.NoSuchAlgorithmException; +import java.security.UnrecoverableKeyException; +import java.security.cert.CertificateException; import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; +import javax.net.ssl.KeyManager; +import javax.net.ssl.KeyManagerFactory; +import javax.net.ssl.SSLContext; +import javax.net.ssl.SSLSocketFactory; +import javax.net.ssl.TrustManager; +import javax.net.ssl.TrustManagerFactory; +import javax.net.ssl.X509TrustManager; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; @@ -39,6 +57,7 @@ import org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcResponse; import org.apache.ignite.internal.util.ipc.loopback.IpcClientTcpEndpoint; +import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteProductVersion; @@ -58,11 +77,8 @@ public class JdbcThinTcpIo { /** Version 2.4.0. */ private static final ClientListenerProtocolVersion VER_2_4_0 = ClientListenerProtocolVersion.create(2, 4, 0); - /** Version 2.5.0. */ - private static final ClientListenerProtocolVersion VER_2_5_0 = ClientListenerProtocolVersion.create(2, 5, 0); - /** Current version. */ - private static final ClientListenerProtocolVersion CURRENT_VER = VER_2_5_0; + private static final ClientListenerProtocolVersion CURRENT_VER = VER_2_4_0; /** Initial output stream capacity for handshake. */ private static final int HANDSHAKE_MSG_SIZE = 13; @@ -166,11 +182,6 @@ public void handshake(ClientListenerProtocolVersion ver) throws IOException, SQL writer.writeBoolean(connProps.isAutoCloseServerCursor()); writer.writeBoolean(connProps.isLazy()); writer.writeBoolean(connProps.isSkipReducerOnUpdate()); - writer.writeBoolean(connProps.isStream()); - writer.writeBoolean(connProps.isStreamAllowOverwrite()); - writer.writeInt(connProps.getStreamParallelOperations()); - writer.writeInt(connProps.getStreamBufferSize()); - writer.writeLong(connProps.getStreamFlushFrequency()); send(writer.array()); @@ -204,8 +215,7 @@ public void handshake(ClientListenerProtocolVersion ver) throws IOException, SQL ClientListenerProtocolVersion srvProtocolVer = ClientListenerProtocolVersion.create(maj, min, maintenance); - if (VER_2_4_0.equals(srvProtocolVer) || VER_2_3_0.equals(srvProtocolVer) || - VER_2_1_5.equals(srvProtocolVer)) + if (VER_2_3_0.equals(srvProtocolVer) || VER_2_1_5.equals(srvProtocolVer)) handshake(srvProtocolVer); else if (VER_2_1_0.equals(srvProtocolVer)) handshake_2_1_0(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java index aaa51a0db6ca7..b51e0b95084ef 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java @@ -60,7 +60,6 @@ import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode; import org.apache.ignite.internal.processors.odbc.SqlStateCode; import org.apache.ignite.internal.processors.query.GridQueryIndexing; -import org.apache.ignite.internal.processors.query.IgniteSQLException; import org.apache.ignite.internal.processors.query.QueryUtils; import org.apache.ignite.internal.processors.resource.GridSpringResourceContext; import org.apache.ignite.internal.util.future.GridFutureAdapter; @@ -83,13 +82,13 @@ import static org.apache.ignite.IgniteJdbcDriver.PROP_LOCAL; import static org.apache.ignite.IgniteJdbcDriver.PROP_MULTIPLE_STMTS; import static org.apache.ignite.IgniteJdbcDriver.PROP_NODE_ID; -import static org.apache.ignite.IgniteJdbcDriver.PROP_SKIP_REDUCER_ON_UPDATE; import static org.apache.ignite.IgniteJdbcDriver.PROP_STREAMING; import static org.apache.ignite.IgniteJdbcDriver.PROP_STREAMING_ALLOW_OVERWRITE; import static org.apache.ignite.IgniteJdbcDriver.PROP_STREAMING_FLUSH_FREQ; import static org.apache.ignite.IgniteJdbcDriver.PROP_STREAMING_PER_NODE_BUF_SIZE; import static org.apache.ignite.IgniteJdbcDriver.PROP_STREAMING_PER_NODE_PAR_OPS; import static org.apache.ignite.IgniteJdbcDriver.PROP_TX_ALLOWED; +import static org.apache.ignite.IgniteJdbcDriver.PROP_SKIP_REDUCER_ON_UPDATE; import static org.apache.ignite.internal.jdbc2.JdbcUtils.convertToSqlException; import static org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode.createJdbcSqlException; @@ -613,11 +612,10 @@ private Ignite getIgnite(String cfgUrl) throws IgniteCheckedException { PreparedStatement nativeStmt = prepareNativeStatement(sql); - try { - idx.checkStatementStreamable(nativeStmt); - } - catch (IgniteSQLException e) { - throw e.toJdbcException(); + if (!idx.isInsertStatement(nativeStmt)) { + throw new SQLException("Only INSERT operations are supported in streaming mode", + SqlStateCode.INTERNAL_ERROR, + IgniteQueryErrorCode.UNSUPPORTED_OPERATION); } IgniteDataStreamer streamer = ignite().dataStreamer(cacheName); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStreamedPreparedStatement.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStreamedPreparedStatement.java index 25f55f2f6d2cd..408f0897cad6f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStreamedPreparedStatement.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStreamedPreparedStatement.java @@ -55,7 +55,7 @@ class JdbcStreamedPreparedStatement extends JdbcPreparedStatement { /** {@inheritDoc} */ @Override protected void execute0(String sql, Boolean isQuery) throws SQLException { - assert isQuery == null || !isQuery; + assert isQuery != null && !isQuery; long updCnt = conn.ignite().context().query().streamUpdateQuery(conn.cacheName(), conn.schemaName(), streamer, sql, getArgs()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java index 735a4decf62f2..7f71c743769bf 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java @@ -571,7 +571,7 @@ private QueryCursor> queryContinuous(ContinuousQuery qry, bool boolean keepBinary = opCtxCall != null && opCtxCall.isKeepBinary(); - return ctx.kernalContext().query().querySqlFields(ctx, qry, null, keepBinary, false); + return ctx.kernalContext().query().querySqlFields(ctx, qry, keepBinary, false); } catch (Exception e) { if (e instanceof CacheException) @@ -604,7 +604,7 @@ private QueryCursor> queryContinuous(ContinuousQuery qry, bool if (qry instanceof SqlFieldsQuery) return (FieldsQueryCursor)ctx.kernalContext().query().querySqlFields(ctx, (SqlFieldsQuery)qry, - null, keepBinary, true).get(0); + keepBinary, true).get(0); if (qry instanceof ScanQuery) return query((ScanQuery)qry, null, projection(qry.isLocal())); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java index 214d006da9956..5841a4d450e9f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java @@ -28,7 +28,7 @@ import org.apache.ignite.internal.util.GridSpinBusyLock; /** - * JDBC Connection Context. + * ODBC Connection Context. */ public class JdbcConnectionContext implements ClientListenerConnectionContext { /** Version 2.1.0. */ @@ -38,16 +38,13 @@ public class JdbcConnectionContext implements ClientListenerConnectionContext { private static final ClientListenerProtocolVersion VER_2_1_5 = ClientListenerProtocolVersion.create(2, 1, 5); /** Version 2.3.1: added "multiple statements query" feature. */ - static final ClientListenerProtocolVersion VER_2_3_0 = ClientListenerProtocolVersion.create(2, 3, 0); + public static final ClientListenerProtocolVersion VER_2_3_0 = ClientListenerProtocolVersion.create(2, 3, 0); /** Version 2.4.0: adds default values for columns feature. */ - static final ClientListenerProtocolVersion VER_2_4_0 = ClientListenerProtocolVersion.create(2, 4, 0); - - /** Version 2.5.0: adds streaming via thin connection. */ - static final ClientListenerProtocolVersion VER_2_5_0 = ClientListenerProtocolVersion.create(2, 5, 0); + public static final ClientListenerProtocolVersion VER_2_4_0 = ClientListenerProtocolVersion.create(2, 4, 0); /** Current version. */ - private static final ClientListenerProtocolVersion CURRENT_VER = VER_2_5_0; + private static final ClientListenerProtocolVersion CURRENT_VER = VER_2_4_0; /** Supported versions. */ private static final Set SUPPORTED_VERS = new HashSet<>(); @@ -69,7 +66,6 @@ public class JdbcConnectionContext implements ClientListenerConnectionContext { static { SUPPORTED_VERS.add(CURRENT_VER); - SUPPORTED_VERS.add(VER_2_4_0); SUPPORTED_VERS.add(VER_2_3_0); SUPPORTED_VERS.add(VER_2_1_5); SUPPORTED_VERS.add(VER_2_1_0); @@ -117,23 +113,8 @@ public JdbcConnectionContext(GridKernalContext ctx, GridSpinBusyLock busyLock, i if (ver.compareTo(VER_2_3_0) >= 0) skipReducerOnUpdate = reader.readBoolean(); - boolean stream = false; - boolean streamAllowOverwrites = false; - int streamParOps = 0; - int streamBufSize = 0; - long streamFlushFreq = 0; - - if (ver.compareTo(VER_2_5_0) >= 0) { - stream = reader.readBoolean(); - streamAllowOverwrites = reader.readBoolean(); - streamParOps = reader.readInt(); - streamBufSize = reader.readInt(); - streamFlushFreq = reader.readLong(); - } - handler = new JdbcRequestHandler(ctx, busyLock, maxCursors, distributedJoins, enforceJoinOrder, - collocated, replicatedOnly, autoCloseCursors, lazyExec, skipReducerOnUpdate, stream, streamAllowOverwrites, - streamParOps, streamBufSize, streamFlushFreq, ver); + collocated, replicatedOnly, autoCloseCursors, lazyExec, skipReducerOnUpdate, ver); parser = new JdbcMessageParser(ctx); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java index 8786f26bf21b1..59fc06bcd02c7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java @@ -53,7 +53,6 @@ import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor; import org.apache.ignite.internal.processors.query.IgniteSQLException; import org.apache.ignite.internal.processors.query.QueryUtils; -import org.apache.ignite.internal.processors.query.SqlClientContext; import org.apache.ignite.internal.util.GridSpinBusyLock; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.X; @@ -89,9 +88,6 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler { /** Kernel context. */ private final GridKernalContext ctx; - /** Client context. */ - private final SqlClientContext cliCtx; - /** Logger. */ private final IgniteLogger log; @@ -107,9 +103,24 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler { /** Current bulk load processors. */ private final ConcurrentHashMap bulkLoadRequests = new ConcurrentHashMap<>(); + /** Distributed joins flag. */ + private final boolean distributedJoins; + + /** Enforce join order flag. */ + private final boolean enforceJoinOrder; + + /** Collocated flag. */ + private final boolean collocated; + /** Replicated only flag. */ private final boolean replicatedOnly; + /** Lazy query execution flag. */ + private final boolean lazy; + + /** Skip reducer on update flag. */ + private final boolean skipReducerOnUpdate; + /** Automatic close of cursors. */ private final boolean autoCloseCursors; @@ -129,38 +140,22 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler { * @param autoCloseCursors Flag to automatically close server cursors. * @param lazy Lazy query execution flag. * @param skipReducerOnUpdate Skip reducer on update flag. - * @param stream Streaming flag. - * @param streamAllowOverwrites Streaming overwrites flag. - * @param streamParOps Number of parallel ops per cluster node during streaming. - * @param streamBufSize Buffer size per cluster node during streaming. - * @param streamFlushFreq Data streamers' flush timeout. * @param protocolVer Protocol version. */ public JdbcRequestHandler(GridKernalContext ctx, GridSpinBusyLock busyLock, int maxCursors, boolean distributedJoins, boolean enforceJoinOrder, boolean collocated, boolean replicatedOnly, boolean autoCloseCursors, boolean lazy, boolean skipReducerOnUpdate, - boolean stream, boolean streamAllowOverwrites, int streamParOps, int streamBufSize, long streamFlushFreq, ClientListenerProtocolVersion protocolVer) { this.ctx = ctx; - - this.cliCtx = new SqlClientContext( - ctx, - distributedJoins, - enforceJoinOrder, - collocated, - lazy, - skipReducerOnUpdate, - stream, - streamAllowOverwrites, - streamParOps, - streamBufSize, - streamFlushFreq - ); - this.busyLock = busyLock; this.maxCursors = maxCursors; + this.distributedJoins = distributedJoins; + this.enforceJoinOrder = enforceJoinOrder; + this.collocated = collocated; this.replicatedOnly = replicatedOnly; this.autoCloseCursors = autoCloseCursors; + this.lazy = lazy; + this.skipReducerOnUpdate = skipReducerOnUpdate; this.protocolVer = protocolVer; log = ctx.log(getClass()); @@ -306,8 +301,6 @@ public void onDisconnect() { } bulkLoadRequests.clear(); - - U.close(cliCtx, log); } finally { busyLock.leaveBusy(); @@ -333,8 +326,6 @@ private JdbcResponse executeQuery(JdbcQueryExecuteRequest req) { long qryId = QRY_ID_GEN.getAndIncrement(); - assert !cliCtx.isStream(); - try { String sql = req.sqlQuery(); @@ -356,17 +347,17 @@ private JdbcResponse executeQuery(JdbcQueryExecuteRequest req) { qry = new SqlFieldsQueryEx(sql, false); - if (cliCtx.isSkipReducerOnUpdate()) + if (skipReducerOnUpdate) ((SqlFieldsQueryEx)qry).setSkipReducerOnUpdate(true); } qry.setArgs(req.arguments()); - qry.setDistributedJoins(cliCtx.isDistributedJoins()); - qry.setEnforceJoinOrder(cliCtx.isEnforceJoinOrder()); - qry.setCollocated(cliCtx.isCollocated()); + qry.setDistributedJoins(distributedJoins); + qry.setEnforceJoinOrder(enforceJoinOrder); + qry.setCollocated(collocated); qry.setReplicatedOnly(replicatedOnly); - qry.setLazy(cliCtx.isLazy()); + qry.setLazy(lazy); if (req.pageSize() <= 0) return new JdbcResponse(IgniteQueryErrorCode.UNKNOWN, "Invalid fetch size: " + req.pageSize()); @@ -380,7 +371,7 @@ private JdbcResponse executeQuery(JdbcQueryExecuteRequest req) { qry.setSchema(schemaName); - List>> results = ctx.query().querySqlFields(null, qry, cliCtx, true, + List>> results = ctx.query().querySqlFields(qry, true, protocolVer.compareTo(VER_2_3_0) < 0); FieldsQueryCursor> fieldsCur = results.get(0); @@ -578,11 +569,11 @@ private ClientListenerResponse executeBatch(JdbcBatchExecuteRequest req) { qry = new SqlFieldsQueryEx(q.sql(), false); - qry.setDistributedJoins(cliCtx.isDistributedJoins()); - qry.setEnforceJoinOrder(cliCtx.isEnforceJoinOrder()); - qry.setCollocated(cliCtx.isCollocated()); + qry.setDistributedJoins(distributedJoins); + qry.setEnforceJoinOrder(enforceJoinOrder); + qry.setCollocated(collocated); qry.setReplicatedOnly(replicatedOnly); - qry.setLazy(cliCtx.isLazy()); + qry.setLazy(lazy); qry.setSchema(schemaName); } @@ -610,21 +601,10 @@ private ClientListenerResponse executeBatch(JdbcBatchExecuteRequest req) { * @param updCntsAcc Per query rows updates counter. * @param firstErr First error data - code and message. */ - @SuppressWarnings("ForLoopReplaceableByForEach") private void executeBatchedQuery(SqlFieldsQueryEx qry, List updCntsAcc, IgniteBiTuple firstErr) { try { - if (cliCtx.isStream()) { - List cnt = ctx.query().streamBatchedUpdateQuery(qry.getSchema(), cliCtx, qry.getSql(), - qry.batchedArguments()); - - for (int i = 0; i < cnt.size(); i++) - updCntsAcc.add(cnt.get(i).intValue()); - - return; - } - - List>> qryRes = ctx.query().querySqlFields(null, qry, cliCtx, true, true); + List>> qryRes = ctx.query().querySqlFields(qry, true, true); for (FieldsQueryCursor> cur : qryRes) { if (cur instanceof BulkLoadContextCursor) 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 dedd0752f8684..6b425a1ce4a58 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 @@ -76,46 +76,33 @@ public QueryCursor> queryDistributedSql(String schemaNa * Detect whether SQL query should be executed in distributed or local manner and execute it. * @param schemaName Schema name. * @param qry Query. - * @param cliCtx Client context. * @param keepBinary Keep binary flag. * @param failOnMultipleStmts Whether an exception should be thrown for multiple statements query. - * @param cancel Query cancel state handler. @return Cursor. + * @param cancel Query cancel state handler. + * @return Cursor. */ - public List>> querySqlFields(String schemaName, SqlFieldsQuery qry, - SqlClientContext cliCtx, boolean keepBinary, boolean failOnMultipleStmts, GridQueryCancel cancel); + public List>> querySqlFields(String schemaName, SqlFieldsQuery qry, boolean keepBinary, + boolean failOnMultipleStmts, GridQueryCancel cancel); /** - * Execute an INSERT statement using data streamer as receiver. + * Perform a MERGE statement using data streamer as receiver. * * @param schemaName Schema name. * @param qry Query. * @param params Query parameters. * @param streamer Data streamer to feed data to. - * @return Update counter. + * @return Query result. * @throws IgniteCheckedException If failed. */ public long streamUpdateQuery(String schemaName, String qry, @Nullable Object[] params, IgniteDataStreamer streamer) throws IgniteCheckedException; - /** - * Execute a batched INSERT statement using data streamer as receiver. - * - * @param schemaName Schema name. - * @param qry Query. - * @param params Query parameters. - * @param cliCtx Client connection context. - * @return Update counters. - * @throws IgniteCheckedException If failed. - */ - public List streamBatchedUpdateQuery(String schemaName, String qry, List params, - SqlClientContext cliCtx) throws IgniteCheckedException; - /** * Executes regular query. * * @param schemaName Schema name. * @param cacheName Cache name. - * @param qry Query. + *@param qry Query. * @param filter Cache name and key filter. * @param keepBinary Keep binary flag. @return Cursor. */ @@ -326,11 +313,12 @@ public void remove(GridCacheContext cctx, GridQueryTypeDescriptor type, CacheDat public String schema(String cacheName); /** - * Check if passed statement is insert statement eligible for streaming, throw an {@link IgniteSQLException} if not. + * Check if passed statement is insert statemtn. * * @param nativeStmt Native statement. + * @return {@code True} if insert. */ - public void checkStatementStreamable(PreparedStatement nativeStmt); + public boolean isInsertStatement(PreparedStatement nativeStmt); /** * Return row cache cleaner. 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 0c73b05edd8db..43bed4026ed2f 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 @@ -64,7 +64,9 @@ import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor; import org.apache.ignite.internal.processors.cache.GridCacheAdapter; 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.StoredCacheData; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.processors.cache.query.CacheQueryFuture; import org.apache.ignite.internal.processors.cache.query.CacheQueryType; @@ -100,7 +102,6 @@ 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.A; 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; @@ -190,9 +191,6 @@ public class GridQueryProcessor extends GridProcessorAdapter { /** Pending status messages. */ private final LinkedList pendingMsgs = new LinkedList<>(); - /** All currently open client contexts. */ - private final Set cliCtxs = Collections.newSetFromMap(new ConcurrentHashMap<>()); - /** Current cache that has a query running on it. */ private final ThreadLocal curCache = new ThreadLocal<>(); @@ -263,15 +261,11 @@ public GridQueryProcessor(GridKernalContext ctx) throws IgniteCheckedException { if (cancel && idx != null) { try { - while (!busyLock.tryBlock(500)) { + while (!busyLock.tryBlock(500)) idx.cancelAllQueries(); - closeAllSqlStreams(); - } - return; - } - catch (InterruptedException ignored) { + } catch (InterruptedException ignored) { U.warn(log, "Interrupted while waiting for active queries cancellation."); Thread.currentThread().interrupt(); @@ -354,32 +348,6 @@ public void onCacheReconnect() throws IgniteCheckedException { } } - /** - * @param cliCtx Client context to register. - */ - void registerClientContext(SqlClientContext cliCtx) { - A.notNull(cliCtx, "cliCtx"); - - cliCtxs.add(cliCtx); - } - - /** - * @param cliCtx Client context to register. - */ - void unregisterClientContext(SqlClientContext cliCtx) { - A.notNull(cliCtx, "cliCtx"); - - cliCtxs.remove(cliCtx); - } - - /** - * Flush streamers on all currently open client contexts. - */ - private void closeAllSqlStreams() { - for (SqlClientContext cliCtx : cliCtxs) - U.close(cliCtx, log); - } - /** * Process schema propose message from discovery thread. * @@ -2008,7 +1976,13 @@ private void checkxEnabled() throws IgniteException { */ public List>> querySqlFields(final SqlFieldsQuery qry, final boolean keepBinary, final boolean failOnMultipleStmts) { - return querySqlFields(null, qry, null, keepBinary, failOnMultipleStmts); + return querySqlFields(null, qry, keepBinary, failOnMultipleStmts); + } + + @SuppressWarnings("unchecked") + public FieldsQueryCursor> querySqlFields(final GridCacheContext cctx, final SqlFieldsQuery qry, + final boolean keepBinary) { + return querySqlFields(cctx, qry, keepBinary, true).get(0); } /** @@ -2019,7 +1993,7 @@ public List>> querySqlFields(final SqlFieldsQuery qry, * @return Cursor. */ public FieldsQueryCursor> querySqlFields(final SqlFieldsQuery qry, final boolean keepBinary) { - return querySqlFields(null, qry, null, keepBinary, true).get(0); + return querySqlFields(null, qry, keepBinary, true).get(0); } /** @@ -2027,16 +2001,14 @@ public FieldsQueryCursor> querySqlFields(final SqlFieldsQuery qry, final * * @param cctx Cache context. * @param qry Query. - * @param cliCtx Client context. * @param keepBinary Keep binary flag. * @param failOnMultipleStmts If {@code true} the method must throws exception when query contains * more then one SQL statement. * @return Cursor. */ @SuppressWarnings("unchecked") - public List>> querySqlFields(@Nullable final GridCacheContext cctx, - final SqlFieldsQuery qry, final SqlClientContext cliCtx, final boolean keepBinary, - final boolean failOnMultipleStmts) { + public List>> querySqlFields(@Nullable final GridCacheContext cctx, + final SqlFieldsQuery qry, final boolean keepBinary, final boolean failOnMultipleStmts) { checkxEnabled(); validateSqlFieldsQuery(qry); @@ -2064,7 +2036,7 @@ public List>> querySqlFields(@Nullable final GridCache GridQueryCancel cancel = new GridQueryCancel(); List>> res = - idx.querySqlFields(schemaName, qry, cliCtx, keepBinary, failOnMultipleStmts, cancel); + idx.querySqlFields(schemaName, qry, keepBinary, failOnMultipleStmts, cancel); if (cctx != null) sendQueryExecutedEvent(qry.getSql(), qry.getArgs(), cctx.name()); @@ -2103,7 +2075,7 @@ private static void validateSqlFieldsQuery(SqlFieldsQuery qry) { * @param schemaName Schema name. * @param streamer Data streamer. * @param qry Query. - * @return Update counter. + * @return Iterator. */ public long streamUpdateQuery(@Nullable final String cacheName, final String schemaName, final IgniteDataStreamer streamer, final String qry, final Object[] args) { @@ -2129,33 +2101,6 @@ public long streamUpdateQuery(@Nullable final String cacheName, final String sch } } - /** - * @param schemaName Schema name. - * @param cliCtx Client context. - * @param qry Query. - * @param args Query arguments. - * @return Update counters. - */ - public List streamBatchedUpdateQuery(final String schemaName, final SqlClientContext cliCtx, - final String qry, final List args) { - if (!busyLock.enterBusy()) - throw new IllegalStateException("Failed to execute query (grid is stopping)."); - - try { - return executeQuery(GridCacheQueryType.SQL_FIELDS, qry, null, new IgniteOutClosureX>() { - @Override public List applyx() throws IgniteCheckedException { - return idx.streamBatchedUpdateQuery(schemaName, qry, args, cliCtx); - } - }, true); - } - catch (IgniteCheckedException e) { - throw new CacheException(e); - } - finally { - busyLock.leaveBusy(); - } - } - /** * Execute distributed SQL query. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/SqlClientContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/SqlClientContext.java deleted file mode 100644 index b1855355cab01..0000000000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/SqlClientContext.java +++ /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. - */ - -package org.apache.ignite.internal.processors.query; - -import java.util.HashMap; -import java.util.Map; -import org.apache.ignite.IgniteDataStreamer; -import org.apache.ignite.IgniteLogger; -import org.apache.ignite.internal.GridKernalContext; -import org.apache.ignite.internal.util.typedef.internal.U; - -/** - * Container for connection properties passed by various drivers (JDBC drivers, possibly ODBC) having notion of an - * SQL connection - Ignite basically does not have one.

- * Also contains anything that a driver may need to share between threads processing queries of logically same client - - * see JDBC thin driver - */ -public class SqlClientContext implements AutoCloseable { - /** Kernal context. */ - private final GridKernalContext ctx; - - /** Distributed joins flag. */ - private final boolean distributedJoins; - - /** Enforce join order flag. */ - private final boolean enforceJoinOrder; - - /** Collocated flag. */ - private final boolean collocated; - - /** Lazy query execution flag. */ - private final boolean lazy; - - /** Skip reducer on update flag. */ - private final boolean skipReducerOnUpdate; - - /** Allow overwrites for duplicate keys on streamed {@code INSERT}s. */ - private final boolean streamAllowOverwrite; - - /** Parallel ops count per node for data streamer. */ - private final int streamNodeParOps; - - /** Node buffer size for data streamer. */ - private final int streamNodeBufSize; - - /** Auto flush frequency for streaming. */ - private final long streamFlushTimeout; - - /** Streamers for various caches. */ - private final Map> streamers; - - /** Logger. */ - private final IgniteLogger log; - - /** - * @param ctx Kernal context. - * @param distributedJoins Distributed joins flag. - * @param enforceJoinOrder Enforce join order flag. - * @param collocated Collocated flag. - * @param lazy Lazy query execution flag. - * @param skipReducerOnUpdate Skip reducer on update flag. - * @param stream Streaming state flag - * @param streamAllowOverwrite Allow overwrites for duplicate keys on streamed {@code INSERT}s. - * @param streamNodeParOps Parallel ops count per node for data streamer. - * @param streamNodeBufSize Node buffer size for data streamer. - * @param streamFlushTimeout Auto flush frequency for streaming. - */ - public SqlClientContext(GridKernalContext ctx, boolean distributedJoins, boolean enforceJoinOrder, - boolean collocated, boolean lazy, boolean skipReducerOnUpdate, boolean stream, boolean streamAllowOverwrite, - int streamNodeParOps, int streamNodeBufSize, long streamFlushTimeout) { - this.ctx = ctx; - this.distributedJoins = distributedJoins; - this.enforceJoinOrder = enforceJoinOrder; - this.collocated = collocated; - this.lazy = lazy; - this.skipReducerOnUpdate = skipReducerOnUpdate; - this.streamAllowOverwrite = streamAllowOverwrite; - this.streamNodeParOps = streamNodeParOps; - this.streamNodeBufSize = streamNodeBufSize; - this.streamFlushTimeout = streamFlushTimeout; - - streamers = stream ? new HashMap<>() : null; - - log = ctx.log(SqlClientContext.class.getName()); - - ctx.query().registerClientContext(this); - } - - /** - * @return Collocated flag. - */ - public boolean isCollocated() { - return collocated; - } - - /** - * @return Distributed joins flag. - */ - public boolean isDistributedJoins() { - return distributedJoins; - } - - /** - * @return Enforce join order flag. - */ - public boolean isEnforceJoinOrder() { - return enforceJoinOrder; - } - - /** - * @return Lazy query execution flag. - */ - public boolean isLazy() { - return lazy; - } - - /** - * @return Skip reducer on update flag, - */ - public boolean isSkipReducerOnUpdate() { - return skipReducerOnUpdate; - } - - /** - * @return Streaming state flag (on or off). - */ - public boolean isStream() { - return streamers != null; - } - - /** - * @param cacheName Cache name. - * @return Streamer for given cache. - */ - public IgniteDataStreamer streamerForCache(String cacheName) { - Map> curStreamers = streamers; - - if (curStreamers == null) - return null; - - IgniteDataStreamer res = curStreamers.get(cacheName); - - if (res != null) - return res; - - res = ctx.grid().dataStreamer(cacheName); - - IgniteDataStreamer exStreamer = curStreamers.putIfAbsent(cacheName, res); - - if (exStreamer == null) { - res.autoFlushFrequency(streamFlushTimeout); - - res.allowOverwrite(streamAllowOverwrite); - - if (streamNodeBufSize > 0) - res.perNodeBufferSize(streamNodeBufSize); - - if (streamNodeParOps > 0) - res.perNodeParallelOperations(streamNodeParOps); - - return res; - } - else { // Someone got ahead of us. - res.close(); - - return exStreamer; - } - } - - /** {@inheritDoc} */ - @Override public void close() throws Exception { - ctx.query().unregisterClientContext(this); - - if (streamers == null) - return; - - for (IgniteDataStreamer s : streamers.values()) - U.close(s, log); - } -} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlParser.java b/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlParser.java index 70f72a7122bf8..0627defee6ae5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlParser.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlParser.java @@ -122,7 +122,7 @@ private SqlCommand nextCommand0() { return cmd; } else - throw errorUnexpectedToken(lex, CREATE, DROP, ALTER, COPY); + throw errorUnexpectedToken(lex, CREATE, DROP, COPY, ALTER); case QUOTED: case MINUS: diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java index 6d7e9ae0cbdea..df27c5fa59ba5 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java @@ -52,7 +52,6 @@ import org.apache.ignite.internal.processors.query.GridRunningQueryInfo; import org.apache.ignite.internal.processors.query.QueryField; import org.apache.ignite.internal.processors.query.QueryIndexDescriptorImpl; -import org.apache.ignite.internal.processors.query.SqlClientContext; import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitor; import org.apache.ignite.internal.util.GridSpinBusyLock; import org.apache.ignite.internal.util.lang.GridCloseableIterator; @@ -245,18 +244,12 @@ private static class FailedIndexing implements GridQueryIndexing { /** {@inheritDoc} */ @Override public List>> querySqlFields(String schemaName, SqlFieldsQuery qry, - SqlClientContext cliCtx, boolean keepBinary, boolean failOnMultipleStmts, GridQueryCancel cancel) { + boolean keepBinary, boolean failOnMultipleStmts, GridQueryCancel cancel) { return null; } /** {@inheritDoc} */ - @Override public List streamBatchedUpdateQuery(String schemaName, String qry, List params, - SqlClientContext cliCtx) throws IgniteCheckedException { - return Collections.emptyList(); - } - - /** {@inheritDoc} */ - @Override public long streamUpdateQuery(String schemaName, String qry, @Nullable Object[] params, + @Override public long streamUpdateQuery(String spaceName, String qry, @Nullable Object[] params, IgniteDataStreamer streamer) throws IgniteCheckedException { return 0; } @@ -379,8 +372,8 @@ private static class FailedIndexing implements GridQueryIndexing { } /** {@inheritDoc} */ - @Override public void checkStatementStreamable(PreparedStatement nativeStmt) { - // No-op. + @Override public boolean isInsertStatement(PreparedStatement nativeStmt) { + return false; } /** {@inheritDoc} */ 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 8d2a82025e601..b11482880afe5 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 @@ -299,7 +299,7 @@ public static Throwable assertThrows(@Nullable IgniteLogger log, Callable cal call.call(); } catch (Throwable e) { - if (cls != e.getClass() && !cls.isAssignableFrom(e.getClass())) { + if (cls != e.getClass()) { if (e.getClass() == CacheException.class && e.getCause() != null && e.getCause().getClass() == cls) e = e.getCause(); else { diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java index 62dbd507b0880..ea6c7c1a4a4e6 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java @@ -43,10 +43,10 @@ import org.apache.ignite.cache.query.FieldsQueryCursor; import org.apache.ignite.cache.query.SqlFieldsQuery; import org.apache.ignite.internal.GridKernalContext; -import org.apache.ignite.internal.processors.bulkload.BulkLoadAckClientParameters; import org.apache.ignite.internal.processors.bulkload.BulkLoadCacheWriter; -import org.apache.ignite.internal.processors.bulkload.BulkLoadParser; import org.apache.ignite.internal.processors.bulkload.BulkLoadProcessor; +import org.apache.ignite.internal.processors.bulkload.BulkLoadAckClientParameters; +import org.apache.ignite.internal.processors.bulkload.BulkLoadParser; import org.apache.ignite.internal.processors.bulkload.BulkLoadStreamerWriter; import org.apache.ignite.internal.processors.cache.CacheOperationContext; import org.apache.ignite.internal.processors.cache.GridCacheContext; @@ -385,7 +385,6 @@ GridQueryFieldsResult updateSqlFieldsLocal(String schemaName, Connection conn, P /** * Perform given statement against given data streamer. Only rows based INSERT is supported. * - * @param schemaName Schema name. * @param streamer Streamer to feed data to. * @param stmt Statement. * @param args Statement arguments. @@ -393,74 +392,81 @@ GridQueryFieldsResult updateSqlFieldsLocal(String schemaName, Connection conn, P * @throws IgniteCheckedException if failed. */ @SuppressWarnings({"unchecked", "ConstantConditions"}) - long streamUpdateQuery(String schemaName, IgniteDataStreamer streamer, PreparedStatement stmt, final Object[] args) + long streamUpdateQuery(IgniteDataStreamer streamer, PreparedStatement stmt, final Object[] args) throws IgniteCheckedException { - idx.checkStatementStreamable(stmt); - Prepared p = GridSqlQueryParser.prepared(stmt); assert p != null; - final UpdatePlan plan = getPlanForStatement(schemaName, null, p, null, true, null); + final UpdatePlan plan = UpdatePlanBuilder.planForStatement(p, true, idx, null, null, null); - assert plan.isLocalSubquery(); + if (!F.eq(streamer.cacheName(), plan.cacheContext().name())) + throw new IgniteSQLException("Cross cache streaming is not supported, please specify cache explicitly" + + " in connection options", IgniteQueryErrorCode.UNSUPPORTED_OPERATION); - final GridCacheContext cctx = plan.cacheContext(); + if (plan.mode() == UpdateMode.INSERT && plan.rowCount() > 0) { + assert plan.isLocalSubquery(); - QueryCursorImpl> cur; + final GridCacheContext cctx = plan.cacheContext(); - final ArrayList> data = new ArrayList<>(plan.rowCount()); + QueryCursorImpl> cur; - QueryCursorImpl> stepCur = new QueryCursorImpl<>(new Iterable>() { - @Override public Iterator> iterator() { - try { - Iterator> it; + final ArrayList> data = new ArrayList<>(plan.rowCount()); + + QueryCursorImpl> stepCur = new QueryCursorImpl<>(new Iterable>() { + @Override public Iterator> iterator() { + try { + Iterator> it; - if (!F.isEmpty(plan.selectQuery())) { - GridQueryFieldsResult res = idx.queryLocalSqlFields(idx.schema(cctx.name()), - plan.selectQuery(), F.asList(U.firstNotNull(args, X.EMPTY_OBJECT_ARRAY)), - null, false, 0, null); + if (!F.isEmpty(plan.selectQuery())) { + GridQueryFieldsResult res = idx.queryLocalSqlFields(idx.schema(cctx.name()), + plan.selectQuery(), F.asList(U.firstNotNull(args, X.EMPTY_OBJECT_ARRAY)), + null, false, 0, null); - it = res.iterator(); - } - else - it = plan.createRows(U.firstNotNull(args, X.EMPTY_OBJECT_ARRAY)).iterator(); + it = res.iterator(); + } + else + it = plan.createRows(U.firstNotNull(args, X.EMPTY_OBJECT_ARRAY)).iterator(); - return new GridQueryCacheObjectsIterator(it, idx.objectContext(), cctx.keepBinary()); - } - catch (IgniteCheckedException e) { - throw new IgniteException(e); + return new GridQueryCacheObjectsIterator(it, idx.objectContext(), cctx.keepBinary()); + } + catch (IgniteCheckedException e) { + throw new IgniteException(e); + } } - } - }, null); + }, null); - data.addAll(stepCur.getAll()); + data.addAll(stepCur.getAll()); - cur = new QueryCursorImpl<>(new Iterable>() { - @Override public Iterator> iterator() { - return data.iterator(); - } - }, null); + cur = new QueryCursorImpl<>(new Iterable>() { + @Override public Iterator> iterator() { + return data.iterator(); + } + }, null); - if (plan.rowCount() == 1) { - IgniteBiTuple t = plan.processRow(cur.iterator().next()); + if (plan.rowCount() == 1) { + IgniteBiTuple t = plan.processRow(cur.iterator().next()); - streamer.addData(t.getKey(), t.getValue()); + streamer.addData(t.getKey(), t.getValue()); - return 1; - } + return 1; + } - Map rows = new LinkedHashMap<>(plan.rowCount()); + Map rows = new LinkedHashMap<>(plan.rowCount()); - for (List row : cur) { - final IgniteBiTuple t = plan.processRow(row); + for (List row : cur) { + final IgniteBiTuple t = plan.processRow(row); - rows.put(t.getKey(), t.getValue()); - } + rows.put(t.getKey(), t.getValue()); + } - streamer.addData(rows); + streamer.addData(rows); - return rows.size(); + return rows.size(); + } + else + throw new IgniteSQLException("Only tuple based INSERT statements are supported in streaming mode", + IgniteQueryErrorCode.UNSUPPORTED_OPERATION); } /** @@ -513,7 +519,7 @@ private UpdateResult executeUpdateStatement(String schemaName, final GridCacheCo .setPageSize(fieldsQry.getPageSize()) .setTimeout(fieldsQry.getTimeout(), TimeUnit.MILLISECONDS); - cur = (QueryCursorImpl>)idx.querySqlFields(schemaName, newFieldsQry, null, true, true, + cur = (QueryCursorImpl>)idx.querySqlFields(schemaName, newFieldsQry, true, true, cancel).get(0); } else if (plan.hasRows()) @@ -604,7 +610,7 @@ private UpdateResult processDmlSelectResult(GridCacheContext cctx, UpdatePlan pl * @return Update plan. */ @SuppressWarnings({"unchecked", "ConstantConditions"}) - UpdatePlan getPlanForStatement(String schema, Connection conn, Prepared p, SqlFieldsQuery fieldsQry, + private UpdatePlan getPlanForStatement(String schema, Connection conn, Prepared p, SqlFieldsQuery fieldsQry, boolean loc, @Nullable Integer errKeysPos) throws IgniteCheckedException { H2CachedStatementKey planKey = H2CachedStatementKey.forDmlStatement(schema, p.getSQL(), fieldsQry, 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 50b148ba4492c..06c936b9867f6 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 @@ -92,7 +92,6 @@ import org.apache.ignite.internal.processors.query.QueryField; import org.apache.ignite.internal.processors.query.QueryIndexDescriptorImpl; import org.apache.ignite.internal.processors.query.QueryUtils; -import org.apache.ignite.internal.processors.query.SqlClientContext; import org.apache.ignite.internal.processors.query.h2.database.H2RowFactory; import org.apache.ignite.internal.processors.query.h2.database.H2TreeIndex; import org.apache.ignite.internal.processors.query.h2.database.io.H2ExtrasInnerIO; @@ -101,7 +100,6 @@ import org.apache.ignite.internal.processors.query.h2.database.io.H2LeafIO; import org.apache.ignite.internal.processors.query.h2.ddl.DdlStatementsProcessor; import org.apache.ignite.internal.processors.query.h2.dml.DmlUtils; -import org.apache.ignite.internal.processors.query.h2.dml.UpdatePlan; import org.apache.ignite.internal.processors.query.h2.opt.GridH2DefaultTableEngine; import org.apache.ignite.internal.processors.query.h2.opt.GridH2IndexBase; import org.apache.ignite.internal.processors.query.h2.opt.GridH2PlainRowFactory; @@ -122,8 +120,8 @@ import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor; import org.apache.ignite.internal.sql.SqlParseException; import org.apache.ignite.internal.sql.SqlParser; -import org.apache.ignite.internal.sql.command.SqlAlterTableCommand; import org.apache.ignite.internal.sql.command.SqlBulkLoadCommand; +import org.apache.ignite.internal.sql.command.SqlAlterTableCommand; import org.apache.ignite.internal.sql.command.SqlCommand; import org.apache.ignite.internal.sql.command.SqlCreateIndexCommand; import org.apache.ignite.internal.sql.command.SqlDropIndexCommand; @@ -151,6 +149,7 @@ import org.h2.api.ErrorCode; import org.h2.api.JavaObjectSerializer; import org.h2.command.Prepared; +import org.h2.command.dml.Insert; import org.h2.command.dml.NoOperation; import org.h2.engine.Session; import org.h2.engine.SysProperties; @@ -193,7 +192,7 @@ @SuppressWarnings({"UnnecessaryFullyQualifiedName", "NonFinalStaticVariableUsedInClassInitialization"}) public class IgniteH2Indexing implements GridQueryIndexing { public static final Pattern INTERNAL_CMD_RE = Pattern.compile( - "^(create|drop)\\s+index|^alter\\s+table|^copy|^set|^flush", Pattern.CASE_INSENSITIVE); + "^(create|drop)\\s+index|^alter\\s+table|^copy", Pattern.CASE_INSENSITIVE); /* * Register IO for indexes. @@ -502,10 +501,10 @@ private PreparedStatement prepare0(Connection c, String sql) throws SQLException } /** {@inheritDoc} */ - @Override public PreparedStatement prepareNativeStatement(String schemaName, String sql) { + @Override public PreparedStatement prepareNativeStatement(String schemaName, String sql) throws SQLException { Connection conn = connectionForSchema(schemaName); - return prepareStatementAndCaches(conn, sql); + return prepareStatement(conn, sql, true); } /** @@ -1015,60 +1014,7 @@ else if (DdlStatementsProcessor.isDdlStatement(p)) throw new IgniteSQLException(e); } - return dmlProc.streamUpdateQuery(schemaName, streamer, stmt, params); - } - - /** {@inheritDoc} */ - @SuppressWarnings("ForLoopReplaceableByForEach") - @Override public List streamBatchedUpdateQuery(String schemaName, String qry, List params, - SqlClientContext cliCtx) throws IgniteCheckedException { - if (cliCtx == null || !cliCtx.isStream()) { - U.warn(log, "Connection is not in streaming mode."); - - return zeroBatchedStreamedUpdateResult(params.size()); - } - - final Connection conn = connectionForSchema(schemaName); - - final PreparedStatement stmt = prepareStatementAndCaches(conn, qry); - - if (GridSqlQueryParser.checkMultipleStatements(stmt)) - throw new IgniteSQLException("Multiple statements queries are not supported for streaming mode.", - IgniteQueryErrorCode.UNSUPPORTED_OPERATION); - - checkStatementStreamable(stmt); - - Prepared p = GridSqlQueryParser.prepared(stmt); - - UpdatePlan plan = dmlProc.getPlanForStatement(schemaName, conn, p, null, true, null); - - IgniteDataStreamer streamer = cliCtx.streamerForCache(plan.cacheContext().name()); - - if (streamer != null) { - List res = new ArrayList<>(params.size()); - - for (int i = 0; i < params.size(); i++) - res.add(dmlProc.streamUpdateQuery(schemaName, streamer, stmt, params.get(i))); - - return res; - } - else { - U.warn(log, "Streaming has been turned off by concurrent command."); - - return zeroBatchedStreamedUpdateResult(params.size()); - } - } - - /** - * @param size Result size. - * @return List of given size filled with 0Ls. - */ - private static List zeroBatchedStreamedUpdateResult(int size) { - Long[] res = new Long[size]; - - Arrays.fill(res, 0); - - return Arrays.asList(res); + return dmlProc.streamUpdateQuery(streamer, stmt, params); } /** @@ -1454,7 +1400,7 @@ UpdateResult runDistributedUpdate( fqry.setTimeout(qry.getTimeout(), TimeUnit.MILLISECONDS); final QueryCursor> res = - querySqlFields(schemaName, fqry, null, keepBinary, true, null).get(0); + querySqlFields(schemaName, fqry, keepBinary, true, null).get(0); final Iterable> converted = new Iterable>() { @Override public Iterator> iterator() { @@ -1490,19 +1436,19 @@ UpdateResult runDistributedUpdate( * Try executing query using native facilities. * * @param schemaName Schema name. - * @param sql Query. + * @param qry Query. * @return Result or {@code null} if cannot parse/process this query. */ - private List>> tryQueryDistributedSqlFieldsNative(String schemaName, String sql) { + private List>> tryQueryDistributedSqlFieldsNative(String schemaName, SqlFieldsQuery qry) { // Heuristic check for fast return. - if (!INTERNAL_CMD_RE.matcher(sql.trim()).find()) + if (!INTERNAL_CMD_RE.matcher(qry.getSql().trim()).find()) return null; // Parse. SqlCommand cmd; try { - SqlParser parser = new SqlParser(schemaName, sql); + SqlParser parser = new SqlParser(schemaName, qry.getSql()); cmd = parser.nextCommand(); @@ -1510,20 +1456,15 @@ private List>> tryQueryDistributedSqlFieldsNative(Stri if (parser.nextCommand() != null) return null; - // Currently supported commands are: - // CREATE/DROP INDEX - // COPY - // ALTER TABLE - // SET STREAMING - // FLUSH STREAMER + // Currently supported commands are: CREATE/DROP INDEX/COPY/ALTER TABLE if (!(cmd instanceof SqlCreateIndexCommand || cmd instanceof SqlDropIndexCommand || - cmd instanceof SqlAlterTableCommand || cmd instanceof SqlBulkLoadCommand)) + cmd instanceof SqlBulkLoadCommand || cmd instanceof SqlAlterTableCommand)) return null; } catch (Exception e) { // Cannot parse, return. if (log.isDebugEnabled()) - log.debug("Failed to parse SQL with native parser [qry=" + sql + ", err=" + e + ']'); + log.debug("Failed to parse SQL with native parser [qry=" + qry.getSql() + ", err=" + e + ']'); if (!IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_SQL_PARSER_DISABLE_H2_FALLBACK)) return null; @@ -1533,24 +1474,24 @@ private List>> tryQueryDistributedSqlFieldsNative(Stri if (e instanceof SqlParseException) code = ((SqlParseException)e).code(); - throw new IgniteSQLException("Failed to parse DDL statement: " + sql + ": " + e.getMessage(), + throw new IgniteSQLException("Failed to parse DDL statement: " + qry.getSql() + ": " + e.getMessage(), code, e); } // Execute. if (cmd instanceof SqlBulkLoadCommand) { - FieldsQueryCursor> cursor = dmlProc.runNativeDmlStatement(sql, cmd); + FieldsQueryCursor> cursor = dmlProc.runNativeDmlStatement(qry.getSql(), cmd); return Collections.singletonList(cursor); } else { try { - FieldsQueryCursor> cursor = ddlProc.runDdlStatement(sql, cmd); + FieldsQueryCursor> cursor = ddlProc.runDdlStatement(qry.getSql(), cmd); return Collections.singletonList(cursor); } catch (IgniteCheckedException e) { - throw new IgniteSQLException("Failed to execute DDL statement [stmt=" + sql + "]: " + throw new IgniteSQLException("Failed to execute DDL statement [stmt=" + qry.getSql() + "]: " + e.getMessage(), e); } } @@ -1574,8 +1515,8 @@ private void checkQueryType(SqlFieldsQuery qry, boolean isQry) { /** {@inheritDoc} */ @SuppressWarnings("StringEquality") @Override public List>> querySqlFields(String schemaName, SqlFieldsQuery qry, - SqlClientContext cliCtx, boolean keepBinary, boolean failOnMultipleStmts, GridQueryCancel cancel) { - List>> res = tryQueryDistributedSqlFieldsNative(schemaName, qry.getSql()); + boolean keepBinary, boolean failOnMultipleStmts, GridQueryCancel cancel) { + List>> res = tryQueryDistributedSqlFieldsNative(schemaName, qry); if (res != null) return res; @@ -1613,8 +1554,8 @@ private void checkQueryType(SqlFieldsQuery qry, boolean isQry) { // We may use this cached statement only for local queries and non queries. if (qry.isLocal() || !prepared.isQuery()) - return (List>>)doRunPrepared(schemaName, prepared, qry, null, cliCtx, - null, keepBinary, cancel); + return (List>>)doRunPrepared(schemaName, prepared, qry, null, null, + keepBinary, cancel); } } @@ -1644,7 +1585,7 @@ private void checkQueryType(SqlFieldsQuery qry, boolean isQry) { firstArg += prepared.getParameters().size(); - res.addAll(doRunPrepared(schemaName, prepared, newQry, twoStepQry, cliCtx, meta, keepBinary, cancel)); + res.addAll(doRunPrepared(schemaName, prepared, newQry, twoStepQry, meta, keepBinary, cancel)); if (parseRes.twoStepQuery() != null && parseRes.twoStepQueryKey() != null && !parseRes.twoStepQuery().explain()) @@ -1660,14 +1601,14 @@ private void checkQueryType(SqlFieldsQuery qry, boolean isQry) { * @param prepared H2 command. * @param qry Fields query with flags. * @param twoStepQry Two-step query if this query must be executed in a distributed way. - * @param cliCtx Client context, or {@code null} if not applicable. * @param meta Metadata for {@code twoStepQry}. * @param keepBinary Whether binary objects must not be deserialized automatically. - * @param cancel Query cancel state holder. @return Query result. + * @param cancel Query cancel state holder. + * @return Query result. */ private List>> doRunPrepared(String schemaName, Prepared prepared, - SqlFieldsQuery qry, GridCacheTwoStepQuery twoStepQry, @Nullable SqlClientContext cliCtx, - List meta, boolean keepBinary, GridQueryCancel cancel) { + SqlFieldsQuery qry, GridCacheTwoStepQuery twoStepQry, List meta, boolean keepBinary, + GridQueryCancel cancel) { String sqlQry = qry.getSql(); boolean loc = qry.isLocal(); @@ -2336,10 +2277,10 @@ private Collection tables(String cacheName) { } /** {@inheritDoc} */ - @Override public void checkStatementStreamable(PreparedStatement nativeStmt) { - if (!GridSqlQueryParser.isStreamableInsertStatement(nativeStmt)) - throw new IgniteSQLException("Only tuple based INSERT statements are supported in streaming mode.", - IgniteQueryErrorCode.UNSUPPORTED_OPERATION); + @Override public boolean isInsertStatement(PreparedStatement nativeStmt) { + Prepared prep = GridSqlQueryParser.prepared(nativeStmt); + + return prep instanceof Insert; } /** {@inheritDoc} */ diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java index 5441e369b1c8c..6f5b51f1a3651 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java @@ -106,7 +106,7 @@ public void start(final GridKernalContext ctx, IgniteH2Indexing idx) { * @throws IgniteCheckedException On error. */ @SuppressWarnings("unchecked") - public FieldsQueryCursor> runDdlStatement(String sql, SqlCommand cmd) throws IgniteCheckedException { + public FieldsQueryCursor> runDdlStatement(String sql, SqlCommand cmd) throws IgniteCheckedException{ IgniteInternalFuture fut; try { @@ -211,7 +211,12 @@ else if (cmd instanceof SqlAlterTableCommand) { if (fut != null) fut.get(); - return zeroCursor(); + QueryCursorImpl> resCur = (QueryCursorImpl>)new QueryCursorImpl(Collections.singletonList + (Collections.singletonList(0L)), null, false); + + resCur.fieldsMeta(UPDATE_RESULT_META); + + return resCur; } catch (SchemaOperationException e) { throw convert(e); @@ -224,19 +229,6 @@ else if (cmd instanceof SqlAlterTableCommand) { } } - /** - * @return Single-column, single-row cursor with 0 as number of updated records. - */ - @SuppressWarnings("unchecked") - public static QueryCursorImpl> zeroCursor() { - QueryCursorImpl> resCur = (QueryCursorImpl>)new QueryCursorImpl(Collections.singletonList - (Collections.singletonList(0L)), null, false); - - resCur.fieldsMeta(UPDATE_RESULT_META); - - return resCur; - } - /** * Execute DDL statement. * diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java index 98fbb97eb80ad..10d485a84c761 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java @@ -505,7 +505,7 @@ public String selectQuery() { /** * @return Local subquery flag. */ - public boolean isLocalSubquery() { + @Nullable public boolean isLocalSubquery() { return isLocSubqry; } } diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java index d897ac7e9b84c..bced83667aff4 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java @@ -85,21 +85,20 @@ private UpdatePlanBuilder() { * @param loc Local query flag. * @param idx Indexing. * @param conn Connection. - * @param fieldsQry Original query. + * @param fieldsQuery Original query. * @return Update plan. */ public static UpdatePlan planForStatement(Prepared prepared, boolean loc, IgniteH2Indexing idx, - @Nullable Connection conn, @Nullable SqlFieldsQuery fieldsQry, @Nullable Integer errKeysPos) + @Nullable Connection conn, @Nullable SqlFieldsQuery fieldsQuery, @Nullable Integer errKeysPos) throws IgniteCheckedException { + assert !prepared.isQuery(); + GridSqlStatement stmt = new GridSqlQueryParser(false).parse(prepared); if (stmt instanceof GridSqlMerge || stmt instanceof GridSqlInsert) - return planForInsert(stmt, loc, idx, conn, fieldsQry); - else if (stmt instanceof GridSqlUpdate || stmt instanceof GridSqlDelete) - return planForUpdate(stmt, loc, idx, conn, fieldsQry, errKeysPos); + return planForInsert(stmt, loc, idx, conn, fieldsQuery); else - throw new IgniteSQLException("Unsupported operation: " + prepared.getSQL(), - IgniteQueryErrorCode.UNSUPPORTED_OPERATION); + return planForUpdate(stmt, loc, idx, conn, fieldsQuery, errKeysPos); } /** diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java index 2d2c25ce643f0..04bc21226e37d 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java @@ -1992,18 +1992,6 @@ private GridSqlElement parseExpression0(Expression expression, boolean calcTypes expression.getClass().getSimpleName() + ']'); } - /** - * Check if passed statement is insert statement eligible for streaming. - * - * @param nativeStmt Native statement. - * @return {@code True} if streamable insert. - */ - public static boolean isStreamableInsertStatement(PreparedStatement nativeStmt) { - Prepared prep = prepared(nativeStmt); - - return prep instanceof Insert && INSERT_QUERY.get((Insert)prep) == null; - } - /** * @param cond Condition. * @param o Object. 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 cf8bb2ebe6b09..069bdd7092d16 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 @@ -140,7 +140,7 @@ public void testTwoStepGroupAndAggregates() throws Exception { SqlFieldsQuery qry = new SqlFieldsQuery("select f.productId, p.name, f.price " + "from FactPurchase f, \"replicated-prod\".DimProduct p where p.id = f.productId "); - for (List o : qryProc.querySqlFields(cache.context(), qry, null, false, true).get(0).getAll()) { + for (List o : qryProc.querySqlFields(cache.context(), qry, false, true).get(0).getAll()) { X.println("___ -> " + o); set1.add((Integer)o.get(0)); @@ -154,7 +154,7 @@ public void testTwoStepGroupAndAggregates() throws Exception { qry = new SqlFieldsQuery("select productId from FactPurchase group by productId"); - for (List o : qryProc.querySqlFields(cache.context(), qry, null, false, true).get(0).getAll()) { + for (List o : qryProc.querySqlFields(cache.context(), qry, false, true).get(0).getAll()) { X.println("___ -> " + o); assertTrue(set0.add((Integer) o.get(0))); @@ -173,7 +173,7 @@ public void testTwoStepGroupAndAggregates() throws Exception { "where p.id = f.productId " + "group by f.productId, p.name"); - for (List o : qryProc.querySqlFields(cache.context(), qry, null, false, true).get(0).getAll()) { + for (List o : qryProc.querySqlFields(cache.context(), qry, false, true).get(0).getAll()) { X.println("___ -> " + o); assertTrue(names.add((String)o.get(0))); @@ -190,7 +190,7 @@ public void testTwoStepGroupAndAggregates() throws Exception { "group by f.productId, p.name " + "having s >= 15"); - for (List o : qryProc.querySqlFields(cache.context(), qry, null, false, true).get(0).getAll()) { + for (List o : qryProc.querySqlFields(cache.context(), qry, false, true).get(0).getAll()) { X.println("___ -> " + o); assertTrue(i(o, 1) >= 15); @@ -203,7 +203,7 @@ public void testTwoStepGroupAndAggregates() throws Exception { qry = new SqlFieldsQuery("select top 3 distinct productId " + "from FactPurchase f order by productId desc "); - for (List o : qryProc.querySqlFields(cache.context(), qry, null, false, true).get(0).getAll()) { + for (List o : qryProc.querySqlFields(cache.context(), qry, false, true).get(0).getAll()) { X.println("___ -> " + o); assertEquals(top--, o.get(0)); @@ -216,7 +216,7 @@ public void testTwoStepGroupAndAggregates() throws Exception { qry = new SqlFieldsQuery("select distinct productId " + "from FactPurchase f order by productId desc limit 2 offset 1"); - for (List o : qryProc.querySqlFields(cache.context(), qry, null, false, true).get(0).getAll()) { + for (List o : qryProc.querySqlFields(cache.context(), qry, false, true).get(0).getAll()) { X.println("___ -> " + o); assertEquals(top--, o.get(0)); @@ -256,13 +256,13 @@ public void testMultiStatement() throws Exception { GridTestUtils.assertThrows(log, new Callable() { @Override public Object call() throws Exception { - qryProc.querySqlFields(cache.context(), qry, null, false, true); + qryProc.querySqlFields(cache.context(), qry, false, true); return null; } }, IgniteSQLException.class, "Multiple statements queries are not supported"); - List>> cursors = qryProc.querySqlFields(cache.context(), qry, null, false, false); + List>> cursors = qryProc.querySqlFields(cache.context(), qry, false, false); assertEquals(2, cursors.size()); @@ -274,7 +274,7 @@ public void testMultiStatement() throws Exception { GridTestUtils.assertThrows(log, new Callable() { @Override public Object call() throws Exception { - qryProc.querySqlFields(cache.context(), qry, null, false, false); + qryProc.querySqlFields(cache.context(), qry, false, false); return null; } From 74a54d23913bd7195c525d8e222b4e4047515843 Mon Sep 17 00:00:00 2001 From: Ilya Lantukh Date: Wed, 28 Feb 2018 10:08:54 +0300 Subject: [PATCH 033/314] IGNITE-7836 Fixed handling of state change message when forceReassignment is false --- .../dht/preloader/GridDhtPartitionsExchangeFuture.java | 9 +++++++++ 1 file changed, 9 insertions(+) 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 22c96f6d98703..8172c7ab222d4 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 @@ -939,6 +939,15 @@ private ExchangeType onClusterStateChangeRequest(boolean crd) { else if (req.activate()) { // TODO: BLT changes on inactive cluster can't be handled easily because persistent storage hasn't been initialized yet. try { + if (!forceAffReassignment) { + // possible only if cluster contains nodes without forceAffReassignment mode + assert firstEventCache().minimumNodeVersion() + .compareToIgnoreTimestamp(FORCE_AFF_REASSIGNMENT_SINCE) < 0 + : firstEventCache().minimumNodeVersion(); + + cctx.affinity().onBaselineTopologyChanged(this, crd); + } + if (CU.isPersistenceEnabled(cctx.kernalContext().config()) && !cctx.kernalContext().clientNode()) cctx.kernalContext().state().onBaselineTopologyChanged(req.baselineTopology(), req.prevBaselineTopologyHistoryItem()); From 2a70ede048f59753061973495f83927f47452d66 Mon Sep 17 00:00:00 2001 From: Andrey Novikov Date: Fri, 19 Jan 2018 10:05:03 +0700 Subject: [PATCH 034/314] IGNITE-6920 Web Console: Create default account for direct-install package. (cherry picked from commit e5005d9) --- modules/web-console/backend/app/mongo.js | 54 +++++++++++++++++++++--- modules/web-console/backend/package.json | 7 +-- modules/web-console/pom.xml | 3 +- 3 files changed, 52 insertions(+), 12 deletions(-) diff --git a/modules/web-console/backend/app/mongo.js b/modules/web-console/backend/app/mongo.js index 189f45bdf28db..f65da03071225 100644 --- a/modules/web-console/backend/app/mongo.js +++ b/modules/web-console/backend/app/mongo.js @@ -1200,17 +1200,57 @@ module.exports.factory = function(settings, mongoose) { mongodRun = helper.run(); return mongodRun - .catch((err) => console.log('Failed to start embedded MongoDB', err)) + .catch((err) => { + console.log('Failed to start embedded MongoDB', err); + + return Promise.reject(err); + }) .then(() => { console.log('Embedded MongoDB successfully started'); - return mongoose.connect(settings.mongoUrl, {server: {poolSize: 4}}); + return mongoose.connect(settings.mongoUrl, {server: {poolSize: 4}}) + .catch((err) => { + console.log('Failed to connect to embedded MongoDB', err); + + return Promise.reject(err); + }); }) - .catch((err) => { - console.log('Failed to connect to embedded MongoDB', err); + .then(() => defineSchema(mongoose)) + .then((mongo) => { + if (settings.packaged) { + return mongo.Account.count() + .then((count) => { + if (count === 0) { + return Promise.all([ + mongo.Account.create({ + _id: '59fc0c25e145c32be0f83b33', + salt: '7b4ccb9e375508a8f87c8f347083ce98cb8785d857dd18208f9a480e992a26bb', + hash: '909d5ed6e0b0a656ef542e2e8e851e9eb00cfb77984e0a6b4597c335d1436a577b3b289601eb8d1f3646e488cd5ea2bbb3e97fcc131cd6a9571407a45b1817bf1af1dd0ccdd070f07733da19e636ff9787369c5f38f86075f78c60809fe4a52288a68ca38aae0ad2bd0cc77b4cae310abf260e9523d361fd9be60e823a7d8e73954ddb18091e668acd3f57baf9fa7db4267e198d829761997a4741734335589ab62793ceb089e8fffe6e5b0e86f332b33a3011ba44e6efd29736f31cbd2b2023e5173baf517f337eb7a4321ea2b67ec827cffa271d26d3f2def93b5efa3ae7e6e327e55feb121ee96b8ff5016527cc7d854a9b49b44c993387c1093705cb26b1802a2e4c1d34508fb93d051d7e5e2e6cc65b6048a999f94c369973b46b204295f0b2f23f8e30723f9e984ddb2c53dcbf0a77a6d0795d44c3ad97a4ae49d6767db9630e2ef76c2069da87088f1400b1292df9bd787122b2cfef1f26a884a298a0bab3d6e6b689381cf6389d2f019e6cd19e82c84048bacfdd1bee946f9d40dda040be426e583abf92529a1c4f032d5058a9799a77e6642312b8d231d79300d5d0d3f74d62797f9d192e8581698e9539812a539ef1b9fbf718f44dd549896ea9449f6ea744586222e5fc29dfcd5eb79e7646ad3d37868f5073833c554853dee6b067bf2bbfab44c011f2de98a8570292f8109b6bde11e3be51075a656c32b521b7', + email: 'admin@admin', + firstName: 'admin', + lastName: 'admin', + company: 'admin', + country: 'United States', + admin: true, + token: 'ruQvlWff09zqoVYyh6WJ', + attempts: 0, + resetPasswordToken: 'O2GWgOkKkhqpDcxjYnSP' + }), + mongo.Space.create({ + _id: '59fc0c26e145c32be0f83b34', + name: 'Personal space', + owner: '59fc0c25e145c32be0f83b33', + usedBy: [], + demo: false + }) + ]); + } + }) + .then(() => mongo) + .catch(() => mongo); + } - return Promise.reject(err); + return mongo; }); - }) - .then(() => defineSchema(mongoose)); + }); }; diff --git a/modules/web-console/backend/package.json b/modules/web-console/backend/package.json index 889c40d7d4a31..0aa56c91aa0e6 100644 --- a/modules/web-console/backend/package.json +++ b/modules/web-console/backend/package.json @@ -8,7 +8,8 @@ "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", - "build": "pkg . --out-path build" + "build": "pkg . --out-path build", + "mongodb-download": "./node_modules/.bin/mongodb-download" }, "author": "", "contributors": [ @@ -71,7 +72,8 @@ "passport-local-mongoose": "4.0.0", "passport.socketio": "3.7.0", "socket.io": "1.7.3", - "uuid": "3.1.0" + "uuid": "3.1.0", + "pkg": "4.2.4" }, "devDependencies": { "chai": "4.1.0", @@ -81,7 +83,6 @@ "mocha": "3.4.2", "mocha-teamcity-reporter": "1.1.1", "mockgoose": "6.0.8", - "pkg": "4.2.4", "supertest": "3.0.0" } } diff --git a/modules/web-console/pom.xml b/modules/web-console/pom.xml index 1ef199269d03c..f398378896d6c 100644 --- a/modules/web-console/pom.xml +++ b/modules/web-console/pom.xml @@ -132,7 +132,6 @@ - org.apache.maven.plugins maven-clean-plugin @@ -235,7 +234,7 @@ - release-web-agent + release-direct-install package single From 2f1ea2cdb76863008d4514f26845457bdeb7d6ed Mon Sep 17 00:00:00 2001 From: Andrey Novikov Date: Fri, 19 Jan 2018 11:35:30 +0700 Subject: [PATCH 035/314] IGNITE-6920 Minor fix. (cherry picked from commit 9cc7cbf) --- modules/web-console/backend/app/mongo.js | 1 + 1 file changed, 1 insertion(+) diff --git a/modules/web-console/backend/app/mongo.js b/modules/web-console/backend/app/mongo.js index f65da03071225..bd03d6ab2a68e 100644 --- a/modules/web-console/backend/app/mongo.js +++ b/modules/web-console/backend/app/mongo.js @@ -1166,6 +1166,7 @@ module.exports.factory = function(settings, mongoose) { // Connect to mongoDB database. return mongoose.connect(settings.mongoUrl, {server: {poolSize: 4}}) + .then(() => defineSchema(mongoose)) .catch(() => { console.log('Failed to connect to local MongoDB, will try to download and start embedded MongoDB'); From 62652f3fb1563ba149dcbccb80928d50b822ff36 Mon Sep 17 00:00:00 2001 From: alexdel Date: Thu, 25 Jan 2018 15:49:28 +0700 Subject: [PATCH 036/314] IGNITE-7064 Web Console: Implemented basic E2E tests. (cherry picked from commit ce96e4f) --- modules/web-console/.dockerignore | 4 + modules/web-console/DEVNOTES.txt | 33 ++++ .../standalone => e2e}/docker-compose.yml | 43 ++-- modules/web-console/e2e/testcafe/Dockerfile | 32 +++ modules/web-console/e2e/testcafe/envtools.js | 186 ++++++++++++++++++ .../e2e/testcafe/fixtures/admin-panel.js | 60 ++++++ .../web-console/e2e/testcafe/fixtures/auth.js | 183 +++++++++++++++++ .../e2e/testcafe/fixtures/menu-smoke.js | 50 +++++ .../e2e/testcafe/fixtures/user-profile.js | 113 +++++++++++ .../testcafe/helpers.js} | 28 +-- modules/web-console/e2e/testcafe/package.json | 47 +++++ modules/web-console/e2e/testcafe/roles.js | 59 ++++++ modules/web-console/e2e/testcafe/testcafe.js | 86 ++++++++ modules/web-console/e2e/testenv/Dockerfile | 84 ++++++++ modules/web-console/e2e/testenv/entrypoint.sh | 21 ++ .../web-console/e2e/testenv/nginx/nginx.conf | 55 ++++++ .../e2e/testenv/nginx/web-console.conf | 62 ++++++ modules/web-console/frontend/package.json | 15 +- .../frontend/public/stylesheets/style.scss | 4 + .../frontend/views/settings/profile.tpl.pug | 2 +- .../web-console/frontend/views/signin.tpl.pug | 4 +- .../views/templates/validation-error.tpl.pug | 2 +- parent/pom.xml | 1 + 23 files changed, 1128 insertions(+), 46 deletions(-) create mode 100644 modules/web-console/.dockerignore rename modules/web-console/{docker/standalone => e2e}/docker-compose.yml (58%) create mode 100644 modules/web-console/e2e/testcafe/Dockerfile create mode 100644 modules/web-console/e2e/testcafe/envtools.js create mode 100644 modules/web-console/e2e/testcafe/fixtures/admin-panel.js create mode 100644 modules/web-console/e2e/testcafe/fixtures/auth.js create mode 100644 modules/web-console/e2e/testcafe/fixtures/menu-smoke.js create mode 100644 modules/web-console/e2e/testcafe/fixtures/user-profile.js rename modules/web-console/{frontend/test/e2e/exampe.test.js => e2e/testcafe/helpers.js} (58%) create mode 100644 modules/web-console/e2e/testcafe/package.json create mode 100644 modules/web-console/e2e/testcafe/roles.js create mode 100644 modules/web-console/e2e/testcafe/testcafe.js create mode 100644 modules/web-console/e2e/testenv/Dockerfile create mode 100644 modules/web-console/e2e/testenv/entrypoint.sh create mode 100644 modules/web-console/e2e/testenv/nginx/nginx.conf create mode 100644 modules/web-console/e2e/testenv/nginx/web-console.conf diff --git a/modules/web-console/.dockerignore b/modules/web-console/.dockerignore new file mode 100644 index 0000000000000..bcac98f1755d1 --- /dev/null +++ b/modules/web-console/.dockerignore @@ -0,0 +1,4 @@ +.git +*Dockerfile* +*docker-compose* +*/node_modules* \ No newline at end of file diff --git a/modules/web-console/DEVNOTES.txt b/modules/web-console/DEVNOTES.txt index dfb017eb3efb8..f7206118f40ed 100644 --- a/modules/web-console/DEVNOTES.txt +++ b/modules/web-console/DEVNOTES.txt @@ -38,3 +38,36 @@ To build direct-install archive from sources run following command in Ignite pro "mvn clean package -pl :ignite-web-agent,:ignite-web-console -am -P web-console -DskipTests=true -DskipClientDocs -Dmaven.javadoc.skip=true" Assembled archive can be found here: `/modules/web-console/target/ignite-web-console-direct-install-*.zip`. + + +End-to-end tests +================ +E2E tests are performed with TestCafe framework - https://testcafe.devexpress.com/. + +To launch tests on your local machine you will need: +1. Install and launch MongoDB. +2. Optionally install Chromium (https://www.chromium.org/getting-involved/download-chromium or https://chromium.woolyss.com). + You may use any other browser, just set 'BROWSERS' constant in 'modules\web-console\e2e\testcafe.js'. +3. In new terminal change directory to 'modules/web-console/e2e/testcafe' folder and execute: "npm install". +4. To start test environment and tests execute: "npm run test". + +During developing tests you may need to run some particular tests without running all suites. +For this case you need to run environment and test separately. +To perform it do the following: +1. Ensure that MongoDB is up and running and all dependencies for backend and frontend are installed. +2. Open directory "modules/web-console/e2e/testcafe" in terminal. Install dependencies for E2E testing with "npm install" command. +3. Execute command "npm run env". This will start backend and frontend environment. +4. Open another terminal window and run command "node testcafe.js" in the same directory. This will run only tests without launching environment. + +Please refer to TestCafe documentation at https://devexpress.github.io/testcafe/documentation/test-api/test-code-structure.html#skipping-tests + upon how to specify which particular test should be run or skipped. + +You can modify the following params with environment variables: +- DB_URL - connection string to test MongoDB. Default: mongodb://localhost/console-e2e +- APP_URL - URL for test environment applications. Default: http://localhost:9001 +- TEAMCITY - Whether to use TeamCity reporter. Default: false (native Testcafe "spec" reporter is used) + +You can run tests in docker: +1. Install docker and docker-compose. +2. Execute in terminal: "docker-compose up --abort-on-container-exit" in directory "modules/web-console/e2e". +3. If you need to cleanup docker container then execute "docker-compose down". diff --git a/modules/web-console/docker/standalone/docker-compose.yml b/modules/web-console/e2e/docker-compose.yml similarity index 58% rename from modules/web-console/docker/standalone/docker-compose.yml rename to modules/web-console/e2e/docker-compose.yml index c6b73d9a1d018..c265237c79546 100644 --- a/modules/web-console/docker/standalone/docker-compose.yml +++ b/modules/web-console/e2e/docker-compose.yml @@ -15,22 +15,27 @@ # limitations under the License. # -webconsole: - image: apacheignite/web-console-standalone - ports: - - 80:80 - 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 - # 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="" +version: '2' +services: + mongodb: + image: 'mongo:latest' + container_name: 'mongodb' + + testenv: + build: + context: '../' + dockerfile: './e2e/testenv/Dockerfile' + environment: + - mongodb_url=mongodb://mongodb:27017/console-e2e + depends_on: + - mongodb + + e2e: + build: './testcafe' + environment: + - DB_URL=mongodb://mongodb:27017/console-e2e + - APP_URL=http://testenv:9001/ + - TEAMCITY=true + depends_on: + - mongodb + - testenv \ No newline at end of file diff --git a/modules/web-console/e2e/testcafe/Dockerfile b/modules/web-console/e2e/testcafe/Dockerfile new file mode 100644 index 0000000000000..e58cbef519750 --- /dev/null +++ b/modules/web-console/e2e/testcafe/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 testcafe/testcafe:latest + +USER 0 + +RUN mkdir -p /opt/testcafe/tests + +WORKDIR /opt/testcafe/tests + +COPY . /opt/testcafe/tests + +RUN npm install --production && \ + npm cache verify --force && \ + rm -rf /tmp/* + +ENTRYPOINT ["node", "./testcafe.js"] \ No newline at end of file diff --git a/modules/web-console/e2e/testcafe/envtools.js b/modules/web-console/e2e/testcafe/envtools.js new file mode 100644 index 0000000000000..846a33a725cab --- /dev/null +++ b/modules/web-console/e2e/testcafe/envtools.js @@ -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. + */ + +const MongoClient = require('mongodb').MongoClient; +const objectid = require('objectid'); +const { spawn } = require('child_process'); +const url = require('url'); + +const argv = require('minimist')(process.argv.slice(2)); +const start = argv._.includes('start'); +const stop = argv._.includes('stop'); + +const mongoUrl = process.env.DB_URL || 'mongodb://localhost/console-e2e'; + +const insertTestUser = ({userId = '000000000000000000000001', token = 'ppw4tPI3JUOGHva8CODO' } = options = {}) => { + return new Promise((res, rej) => { + MongoClient + .connect(mongoUrl, function(err, db) { + if (err) { + rej(); + throw err; + } + + // add user + const user = { + _id: objectid(userId), + salt: 'ca8b49c2eacd498a0973de30c0873c166ed99fa0605981726aedcc85bee17832', + hash: 'c052c87e454cd0875332719e1ce085ccd92bedb73c8f939ba45d387f724da97128280643ad4f841d929d48de802f48f4a27b909d2dc806d957d38a1a4049468ce817490038f00ac1416aaf9f8f5a5c476730b46ea22d678421cd269869d4ba9d194f73906e5d5a4fec5229459e20ebda997fb95298067126f6c15346d886d44b67def03bf3ffe484b2e4fa449985de33a0c12e4e1da4c7d71fe7af5d138433f703d8c7eeebbb3d57f1a89659010a1f1d3cd4fbc524abab07860daabb08f08a28b8bfc64ecde2ea3c103030d0d54fc24d9c02f92ee6b3aa1bcd5c70113ab9a8045faea7dd2dc59ec4f9f69fcf634232721e9fb44012f0e8c8fdf7c6bf642db6867ef8e7877123e1bc78af7604fee2e34ad0191f8b97613ea458e0fca024226b7055e08a4bdb256fabf0a203a1e5b6a6c298fb0c60308569cefba779ce1e41fb971e5d1745959caf524ab0bedafce67157922f9c505cea033f6ed28204791470d9d08d31ce7e8003df8a3a05282d4d60bfe6e2f7de06f4b18377dac0fe764ed683c9b2553e75f8280c748aa166fef6f89190b1c6d369ab86422032171e6f9686de42ac65708e63bf018a043601d85bc5c820c7ad1d51ded32e59cdaa629a3f7ae325bbc931f9f21d90c9204effdbd53721a60c8b180dd8c236133e287a47ccc9e5072eb6593771e435e4d5196d50d6ddb32c226651c6503387895c5ad025f69fd3', + password: 'a', + email: 'a@a', + firstName: 'John', + lastName: 'Doe', + company: 'TestCompany', + country: 'Canada', + admin: true, + token, + attempts: 0, + lastLogin: '2016-06-28T10:41:07.463Z', + resetPasswordToken: '892rnLbEnVp1FP75Jgpi' + }; + db.collection('accounts').insert(user); + + // add spaces + + const spaces = [ + { + _id: objectid('000000000000000000000001'), + name: 'Personal space', + owner: objectid(userId), + demo: false + }, + { + _id: objectid('000000000000000000000002'), + name: 'Demo space', + owner: objectid(userId), + demo: true + } + ]; + db.collection('spaces').insertMany(spaces); + + db.close(); + res(); + + }); + }); +}; + +const removeData = () => { + return new Promise((resolve, reject) => { + MongoClient.connect(mongoUrl, async(err, db) => { + if (err) + return reject(err); + + db.dropDatabase((err) => { + if (err) + return reject(err); + + resolve(); + }); + }); + }); +}; + + +/** + * Spawns a new process using the given command. + * @param command {String} The command to run. + * @param onResolveString {String} Await string in output. + * @param cwd {String} Current working directory of the child process. + * @param env {Object} Environment key-value pairs. + * @return {Promise} + */ +const exec = (command, onResolveString, cwd, env) => { + return new Promise((resolve) => { + env = Object.assign({}, process.env, env, { FORCE_COLOR: true }); + + const [cmd, ...args] = command.split(' '); + + const detached = process.platform !== 'win32'; + + const child = spawn(cmd, args, {cwd, env, detached}); + + if (detached) { + // do something when app is closing + process.on('exit', () => process.kill(-child.pid)); + + // catches ctrl+c event + process.on('SIGINT', () => process.kill(-child.pid)); + + // catches "kill pid" (for example: nodemon restart) + process.on('SIGUSR1', () => process.kill(-child.pid)); + process.on('SIGUSR2', () => process.kill(-child.pid)); + + // catches uncaught exceptions + process.on('uncaughtException', () => process.kill(-child.pid)); + } + + // Pipe error messages to stdout. + child.stderr.on('data', (data) => { + process.stdout.write(data.toString()); + }); + + child.stdout.on('data', (data) => { + process.stdout.write(data.toString()); + + if (data.includes(onResolveString)) + resolve(child); + }); + }); +}; + +const startEnv = () => { + return new Promise(async(resolve) => { + const command = `${process.platform === 'win32' ? 'npm.cmd' : 'npm'} start`; + + let port = 9001; + if (process.env.APP_URL) { + port = parseInt(url.parse(process.env.APP_URL).port) || 80; + } + + const backendInstanceLaunch = exec(command, 'Start listening', '../../backend', {server_port: 3001, mongodb_url: mongoUrl}); + const frontendInstanceLaunch = exec(command, 'Compiled successfully', '../../frontend', {BACKEND_PORT: 3001, PORT: port}); + + console.log('Building backend in progress...'); + await backendInstanceLaunch; + console.log('Building backend done!'); + + console.log('Building frontend in progress...'); + await frontendInstanceLaunch; + console.log('Building frontend done!'); + + resolve(); + }); +}; + +if (start) { + startEnv(); + + process.on('SIGINT', async() => { + await removeData(); + + process.exit(0); + }); +} + +if (stop) { + removeData(); + + console.log('Cleaning done...'); +} + +module.exports = { startEnv, removeData, insertTestUser }; diff --git a/modules/web-console/e2e/testcafe/fixtures/admin-panel.js b/modules/web-console/e2e/testcafe/fixtures/admin-panel.js new file mode 100644 index 0000000000000..39c596e2284bb --- /dev/null +++ b/modules/web-console/e2e/testcafe/fixtures/admin-panel.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. + */ + +const { Selector } = require('testcafe'); +const { removeData, insertTestUser } = require('../envtools'); +const { signIn } = require('../roles'); + +fixture('Checking admin panel') + .page `${process.env.APP_URL || 'http://localhost:9001/'}settings/admin` + .beforeEach(async(t) => { + await t.setNativeDialogHandler(() => true); + await removeData(); + await insertTestUser(); + await signIn(t); + + await t.navigateTo(`${process.env.APP_URL || 'http://localhost:9001/'}settings/admin`); + }) + .after(async() => { + await removeData(); + }); + +test('Testing setting notifications', async(t) => { + await t.click(Selector('button').withAttribute('ng-click', 'ctrl.changeUserNotifications()')); + + await t + .expect(Selector('h4').withText(/.*Set user notifications.*/).exists) + .ok() + .click('.ace_content') + .pressKey('t e s t space m e s s a g e') + .click('#btn-submit'); + + await t + .expect(Selector('div').withText('test message').exists) + .ok(); + + await t.click(Selector('button').withAttribute('ng-click', 'ctrl.changeUserNotifications()')); + + await t + .click('.ace_content') + .pressKey('ctrl+a delete') + .click('#btn-submit'); + + await t + .expect(Selector('div').withText('test message').exists) + .notOk(); +}); diff --git a/modules/web-console/e2e/testcafe/fixtures/auth.js b/modules/web-console/e2e/testcafe/fixtures/auth.js new file mode 100644 index 0000000000000..d6d0226c9abdf --- /dev/null +++ b/modules/web-console/e2e/testcafe/fixtures/auth.js @@ -0,0 +1,183 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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 { Selector, Role } = require('testcafe'); +const { signUp } = require('../roles'); +const { AngularJSSelector } = require('testcafe-angular-selectors'); +const { removeData, insertTestUser } = require('../envtools'); + +fixture('Checking Ignite auth screen') + .page `${process.env.APP_URL || 'http://localhost:9001/'}` + .beforeEach(async(t) => { + await removeData(); + + await t.setNativeDialogHandler(() => true); + await t.useRole(Role.anonymous()); + }) + .after(async() => { + await removeData(); + }); + +test('Testing Ignite signup validation and signup success', async(t) => { + async function checkBtnDisabled() { + const btnDisabled = await t.expect(Selector('#signup').getAttribute('disabled')).ok(); + + const btnNotWorks = await t + .click('#signup') + .expect(Selector('title').innerText).eql('Apache Ignite - Management Tool and Configuration Wizard – Apache Ignite Web Console'); + + return btnDisabled && btnNotWorks; + } + + await t.click(Selector('a').withText('Sign Up')); + + await t + .click(Selector('#signup_email')) + .typeText(Selector('#signup_email'), 'test@test.com'); + await checkBtnDisabled(); + + await t + .typeText(AngularJSSelector.byModel('ui.password'), 'qwerty') + .typeText(AngularJSSelector.byModel('ui_exclude.confirm'), 'qwerty'); + await checkBtnDisabled(); + + await t + .typeText(AngularJSSelector.byModel('ui.firstName'), 'John') + .typeText(AngularJSSelector.byModel('ui.lastName'), 'Doe'); + await checkBtnDisabled(); + + await t + .typeText(AngularJSSelector.byModel('ui.company'), 'DevNull LTD'); + await checkBtnDisabled(); + + await t + .click('#country') + .click(Selector('span').withText('Brazil')); + + // checking passwords confirm dismatch + await t + .click(AngularJSSelector.byModel('ui_exclude.confirm')) + .pressKey('ctrl+a delete') + .typeText(AngularJSSelector.byModel('ui_exclude.confirm'), 'ytrewq'); + await checkBtnDisabled(); + await t + .click(AngularJSSelector.byModel('ui_exclude.confirm')) + .pressKey('ctrl+a delete') + .typeText(AngularJSSelector.byModel('ui_exclude.confirm'), 'qwerty'); + + await t.click('#signup') + .expect(Selector('title').innerText).eql('Basic Configuration – Apache Ignite Web Console'); + +}); + +test('Testing Ignite validation and successful sign in of existing user', async(t) => { + async function checkSignInBtnDisabled() { + const btnDisabled = await t.expect(await Selector('#login').getAttribute('disabled')).ok(); + const btnNotWorks = await t + .click('#login') + .expect(Selector('title').innerText).eql('Apache Ignite - Management Tool and Configuration Wizard – Apache Ignite Web Console'); + + return btnDisabled && btnNotWorks; + } + + await insertTestUser(); + + // checking signin validation + await t + .typeText(AngularJSSelector.byModel('ui.email'), 'test@test.com'); + await checkSignInBtnDisabled(); + + await t + .typeText(AngularJSSelector.byModel('ui.password'), 'b') + .click('#login'); + await t.expect(Selector('#popover-validation-message').withText('Invalid email or password').exists).ok(); + + await t + .click(AngularJSSelector.byModel('ui.email')) + .pressKey('ctrl+a delete') + .typeText(AngularJSSelector.byModel('ui.email'), 'testtest.com'); + await checkSignInBtnDisabled(); + + // checking regular sigin in + await t + .click(AngularJSSelector.byModel('ui.email')) + .pressKey('ctrl+a delete') + .typeText(AngularJSSelector.byModel('ui.email'), 'a@a') + .click(AngularJSSelector.byModel('ui.password')) + .pressKey('ctrl+a delete') + .typeText(AngularJSSelector.byModel('ui.password'), 'a') + .click('#login') + .expect(Selector('title').innerText).eql('Basic Configuration – Apache Ignite Web Console'); + +}); + +test('Forbid Ignite signing up of already existing user', async(t) => { + await insertTestUser(); + + await t.click(Selector('a').withText('Sign Up')); + + await t + .click(Selector('#signup_email')) + .typeText(Selector('#signup_email'), 'a@a') + .typeText(AngularJSSelector.byModel('ui.password'), 'a') + .typeText(AngularJSSelector.byModel('ui_exclude.confirm'), 'a') + .typeText(AngularJSSelector.byModel('ui.firstName'), 'John') + .typeText(AngularJSSelector.byModel('ui.lastName'), 'Doe') + .typeText(AngularJSSelector.byModel('ui.company'), 'DevNull LTD') + .click('#country') + .click(Selector('span').withText('Brazil')) + .click('#signup'); + + await t.expect(Selector('#popover-validation-message').withText('A user with the given username is already registered.').exists).ok(); + +}); + +test('Test Ignite password reset', async(t) => { + await t.click(Selector('#password-forgot-signin')); + + // testing incorrect email + await t + .typeText('#forgot_email', 'testtest') + .expect(await Selector('button').withText('Send it to me').getAttribute('disabled')).ok(); + + // testing handling unknown email password reset + await t + .click(Selector('#forgot_email')) + .pressKey('ctrl+a delete') + .typeText(Selector('#forgot_email'), 'nonexisting@mail.com') + .click(Selector('button').withText('Send it to me')); + + await t.expect(Selector('#popover-validation-message').withText('Account with that email address does not exists!').exists).ok(); + + // testing regular password reset + await t + .click(Selector('#forgot_email')) + .pressKey('ctrl+a delete') + .typeText(Selector('#forgot_email'), 'a@a') + .click(Selector('button').withText('Send it to me')); + + await t.expect(Selector('#popover-validation-message').withText('Account with that email address does not exists!').exists).notOk(); +}); + +test('Testing Ignite loguout', async(t) => { + await signUp(t); + + await t.click(Selector('div').withAttribute('bs-dropdown', 'userbar.items')); + await t + .click(Selector('a').withAttribute('ui-sref', 'logout')) + .expect(Selector('title').innerText).eql('Apache Ignite - Management Tool and Configuration Wizard – Apache Ignite Web Console'); +}); diff --git a/modules/web-console/e2e/testcafe/fixtures/menu-smoke.js b/modules/web-console/e2e/testcafe/fixtures/menu-smoke.js new file mode 100644 index 0000000000000..9ae79b4ee30a9 --- /dev/null +++ b/modules/web-console/e2e/testcafe/fixtures/menu-smoke.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. + */ + +const { Selector } = require('testcafe'); +const { removeData } = require('../envtools'); +const { signUp } = require('../roles'); + +fixture('Checking Ingite main menu') + .page `${process.env.APP_URL || 'http://localhost:9001/'}` + .beforeEach(async(t) => { + await t.setNativeDialogHandler(() => true); + await removeData(); + await signUp(t); + }) + .after(async() => { + await removeData(); + }); + +test('Ingite main menu smoke test', async(t) => { + + await t + .click(Selector('a').withAttribute('ui-sref', 'base.configuration.tabs')) + .expect(Selector('title').innerText) + .eql('Basic Configuration – Apache Ignite Web Console'); + + await t + .click(Selector('a').withText('Queries')) + .expect(Selector('h4').withText('New query notebook').exists) + .ok() + .typeText('#create-notebook', 'Test query') + .click('#copy-btn-confirm'); + + await t + .expect(Selector('span').withText('Connection to Ignite Web Agent is not established').exists) + .ok(); +}); diff --git a/modules/web-console/e2e/testcafe/fixtures/user-profile.js b/modules/web-console/e2e/testcafe/fixtures/user-profile.js new file mode 100644 index 0000000000000..133b37e867fea --- /dev/null +++ b/modules/web-console/e2e/testcafe/fixtures/user-profile.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 { Selector } = require('testcafe'); +const { removeData, insertTestUser } = require('../envtools'); +const { signIn, signUp } = require('../roles'); + +fixture('Checking user profile') + .page `${process.env.APP_URL || 'http://localhost:9001/'}settings/profile` + .beforeEach(async(t) => { + await t.setNativeDialogHandler(() => true); + await removeData(); + await insertTestUser(); + await signIn(t); + + await t.navigateTo(`${process.env.APP_URL || 'http://localhost:9001/'}settings/profile`); + }) + .after(async() => { + await removeData(); + }); + +test('Testing user data change', async(t) => { + + const newUserData = { + firstName: { + selector: '#profile-firstname', + value: 'Richard' + }, + lastName: { + selector: '#profile-lastname', + value: 'Roe' + }, + email: { + selector: '#profile-email', + value: 'r.roe@mail.com' + }, + company: { + selector: '#profile-company', + value: 'New Company' + }, + country: { + selector: '#profile-country', + value: 'Israel' + } + }; + + ['firstName', 'lastName', 'email', 'company'].forEach(async(item) => { + await t + .click(newUserData[item].selector) + .pressKey('ctrl+a delete') + .typeText(newUserData[item].selector, newUserData[item].value); + }); + + await t + .click(newUserData.country.selector) + .click(Selector('span').withText(newUserData.country.value)) + .click(Selector('a').withText('Save')); + + await t.navigateTo(`${process.env.APP_URL || 'http://localhost:9001/'}settings/profile`); + + ['firstName', 'lastName', 'email', 'company'].forEach(async(item) => { + await t + .expect(await Selector(newUserData[item].selector).getAttribute('value')) + .eql(newUserData[item].value); + }); + + await t + .expect(Selector(newUserData.country.selector).innerText) + .eql(newUserData.country.value); +}); + +test('Testing secure token change', async(t) => { + await t.click(Selector('a').withAttribute('ng-click', 'toggleToken()')); + + const currentToken = await Selector('#current-security-token').innerText; + + await t + .click(Selector('i').withAttribute('ng-click', 'generateToken()')) + .expect(Selector('p').withText('Are you sure you want to change security token?').exists) + .ok() + .click('#confirm-btn-ok', {timeout: 5000}); + + await t + .expect(await Selector('#current-security-token').innerText) + .notEql(currentToken); +}); + +test('Testing password change', async(t) => { + await t.click(Selector('a').withAttribute('ng-click', 'togglePassword()')); + + await t + .typeText('#profile_password', 'newPass') + .typeText('#profile_confirm', 'newPass') + .click(Selector('a').withText('Save')); + + await t + .expect(Selector('span').withText('Profile saved.').exists) + .ok(); +}); diff --git a/modules/web-console/frontend/test/e2e/exampe.test.js b/modules/web-console/e2e/testcafe/helpers.js similarity index 58% rename from modules/web-console/frontend/test/e2e/exampe.test.js rename to modules/web-console/e2e/testcafe/helpers.js index 00788bbfbe8dd..8ab4d471e291f 100644 --- a/modules/web-console/frontend/test/e2e/exampe.test.js +++ b/modules/web-console/e2e/testcafe/helpers.js @@ -15,28 +15,14 @@ * limitations under the License. */ -import { suite, test, setup } from 'mocha'; +const { ClientFunction } = require('testcafe'); -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(); +const mouseenterTrigger = ClientFunction((selector = '') => { + return new Promise((resolve) => { + window.jQuery(selector).mouseenter(); - // 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'); + resolve(); }); }); + +module.exports = { mouseenterTrigger }; diff --git a/modules/web-console/e2e/testcafe/package.json b/modules/web-console/e2e/testcafe/package.json new file mode 100644 index 0000000000000..0c8ea90add46f --- /dev/null +++ b/modules/web-console/e2e/testcafe/package.json @@ -0,0 +1,47 @@ +{ + "name": "ignite-web-console-e2e-tests", + "version": "1.0.0", + "description": "E2E tests for Apache Ignite Web console", + "private": true, + "scripts": { + "env": "node envtools.js start", + "test": "node testcafe.js --env=true" + }, + "author": "", + "contributors": [ + { + "name": "", + "email": "" + } + ], + "license": "Apache-2.0", + "keywords": [ + "Apache Ignite Web console" + ], + "homepage": "https://ignite.apache.org/", + "engines": { + "npm": "3.x.x", + "node": "4.x.x" + }, + "os": [ + "darwin", + "linux", + "win32" + ], + "dependencies": { + "app-module-path": "2.2.0", + "cross-env": "5.1.1", + "glob": "7.1.2", + "minimist": "1.2.0", + "mongodb": "2.2.33", + "node-cmd": "3.0.0", + "objectid": "3.2.1", + "path": "0.12.7", + "sinon": "2.3.8", + "testcafe": "0.18.5", + "testcafe-angular-selectors": "0.3.0", + "testcafe-reporter-teamcity": "1.0.9", + "type-detect": "4.0.3", + "util": "0.10.3" + } +} diff --git a/modules/web-console/e2e/testcafe/roles.js b/modules/web-console/e2e/testcafe/roles.js new file mode 100644 index 0000000000000..bfa9f13114867 --- /dev/null +++ b/modules/web-console/e2e/testcafe/roles.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. + */ + +const path = require('path'); +const { Selector } = require('testcafe'); +const { AngularJSSelector } = require('testcafe-angular-selectors'); + +const igniteSignUp = async(t) => { + await t.navigateTo(`${process.env.APP_URL || 'http://localhost:9001/'}`); + + await t.click(Selector('a').withText('Sign Up')); + + await t + .click(Selector('#signup_email')) + .typeText(Selector('#signup_email'), 'a@a') + .typeText(AngularJSSelector.byModel('ui.password'), 'a') + .typeText(AngularJSSelector.byModel('ui_exclude.confirm'), 'a') + .typeText(AngularJSSelector.byModel('ui.firstName'), 'John') + .typeText(AngularJSSelector.byModel('ui.lastName'), 'Doe') + .typeText(AngularJSSelector.byModel('ui.company'), 'DevNull LTD') + .click('#country') + .click(Selector('span').withText('Brazil')) + .click('#signup'); + + // close modal window + await t.click('.modal-header button.close'); +}; + + +const igniteSignIn = async(t) => { + await t.navigateTo(`${process.env.APP_URL || 'http://localhost:9001/'}`); + + await t + .typeText(AngularJSSelector.byModel('ui.email'), 'a@a') + .typeText(AngularJSSelector.byModel('ui.password'), 'a') + .click('#login'); + + // close modal window + await t.click('.modal-header button.close'); +}; + +const signIn = process.env.IGNITE_MODULES ? require(path.join(process.env.IGNITE_MODULES, 'e2e/testcafe/roles.js')).igniteSignIn : igniteSignIn; +const signUp = process.env.IGNITE_MODULES ? require(path.join(process.env.IGNITE_MODULES, 'e2e/testcafe/roles.js')).igniteSignUp : igniteSignUp; + +module.exports = { signUp, signIn }; diff --git a/modules/web-console/e2e/testcafe/testcafe.js b/modules/web-console/e2e/testcafe/testcafe.js new file mode 100644 index 0000000000000..597d29a5f985a --- /dev/null +++ b/modules/web-console/e2e/testcafe/testcafe.js @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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 glob = require('glob'); +const path = require('path'); + +require('app-module-path').addPath(path.join(__dirname, 'node_modules')); +require('app-module-path').addPath(__dirname); + +const argv = require('minimist')(process.argv.slice(2)); +const envEnabled = argv.env; + +const { startEnv, removeData } = require('./envtools'); + +const createTestCafe = require('testcafe'); + +// See all supported browsers at http://devexpress.github.io/testcafe/documentation/using-testcafe/common-concepts/browsers/browser-support.html#locally-installed-browsers +const BROWSERS = ['chromium:headless --no-sandbox']; // For example: ['chrome', 'firefox']; + +let testcafe = null; + +const resolveFixturesPaths = () => { + let fixturesPaths = glob.sync('./fixtures/*.js'); + + if (process.env.IGNITE_MODULES) { + const igniteModulesTestcafe = path.join(process.env.IGNITE_MODULES, 'e2e/testcafe'); + const additionalFixturesPaths = glob.sync(path.join(igniteModulesTestcafe, 'fixtures', '*.js')); + const relativePaths = new Set(additionalFixturesPaths.map((fixturePath) => path.relative(igniteModulesTestcafe, fixturePath))); + + fixturesPaths = fixturesPaths.filter((fixturePath) => !relativePaths.has(path.relative(process.cwd(), fixturePath))).concat(additionalFixturesPaths); + } + + return fixturesPaths; +}; + +createTestCafe('localhost', 1337, 1338) + .then(async(tc) => { + try { + if (envEnabled) + await startEnv(); + + await removeData(); + + testcafe = tc; + + const runner = testcafe.createRunner(); + const reporter = process.env.TEAMCITY ? 'teamcity' : 'spec'; + + console.log('Start E2E testing!'); + + return runner + .src(resolveFixturesPaths()) + .browsers(BROWSERS) + .reporter(reporter) + .run({ skipJsErrors: true }); + } catch (err) { + console.log(err); + + process.exit(1); + } + }) + .then(async(failedCount) => { + console.log('Cleaning after tests...'); + + testcafe.close(); + + if (envEnabled) + await removeData(); + + console.log('Tests failed: ' + failedCount); + + process.exit(0); + }); diff --git a/modules/web-console/e2e/testenv/Dockerfile b/modules/web-console/e2e/testenv/Dockerfile new file mode 100644 index 0000000000000..5a8f24b904a06 --- /dev/null +++ b/modules/web-console/e2e/testenv/Dockerfile @@ -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. +# + +FROM ubuntu:14.04 + +ENV NPM_CONFIG_LOGLEVEL info +ENV NODE_VERSION 8.6.0 + +# Before package list update. +RUN set -ex && \ + for key in \ + 9554F04D7259F04124DE6B476D5A82AC7E37093B \ + 94AE36675C464D64BAFA68DD7434390BDBE9B9C5 \ + FD3A5288F042B6850C66B31F09FE44734EB7990E \ + 71DCFD284A79C3B38668286BC97EC7A07EDE3FC1 \ + DD8F2338BAE7501E3DD5AC78C273792F7D83545D \ + B9AE9905FFD7803F25714661B63B535A4C206CA9 \ + C4F0DFFF4E8C1A8236409D08E73BC641CC11F4C8 \ + 56730D5401028683275BD23C23EFEFE93C4CFFFE \ + ; do \ + gpg --keyserver ha.pool.sks-keyservers.net --recv-keys "$key" || \ + gpg --keyserver pgp.mit.edu --recv-keys "$key" || \ + gpg --keyserver keyserver.pgp.com --recv-keys "$key" ; \ + done + +# Update package list & install. +RUN apt-get update && \ + apt-get install -y nginx-light curl xz-utils git dos2unix + +# 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 install -g pm2 + +# Install frontend & backend apps. +RUN mkdir -p /opt/web-console + +# Copy source. +WORKDIR /opt/web-console +COPY frontend ./frontend +COPY backend ./backend + + +# Install node modules. +RUN cd /opt/web-console/frontend && npm install --no-optional --prod && npm run build +RUN cd /opt/web-console/backend && npm install --only=production --no-optional + +# Returns to base path. +WORKDIR /opt/web-console + +# Copy nginx config. +COPY ./e2e/testenv/nginx/nginx.conf /etc/nginx/nginx.conf +COPY ./e2e/testenv/nginx/web-console.conf /etc/nginx/web-console.conf + +# Setup entrypoint. +COPY ./e2e/testenv/entrypoint.sh . +RUN chmod 755 /opt/web-console/entrypoint.sh && dos2unix /opt/web-console/entrypoint.sh + +# Clean up. +RUN apt-get clean && rm -rf /var/lib/apt/lists/* /tmp/* /var/tmp/* + +EXPOSE 9001 + +ENTRYPOINT ["/opt/web-console/entrypoint.sh"] diff --git a/modules/web-console/e2e/testenv/entrypoint.sh b/modules/web-console/e2e/testenv/entrypoint.sh new file mode 100644 index 0000000000000..f6107a4fe7603 --- /dev/null +++ b/modules/web-console/e2e/testenv/entrypoint.sh @@ -0,0 +1,21 @@ +#!/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. +# + +service nginx start + +cd /opt/web-console/backend && pm2 start ./index.js --no-daemon diff --git a/modules/web-console/e2e/testenv/nginx/nginx.conf b/modules/web-console/e2e/testenv/nginx/nginx.conf new file mode 100644 index 0000000000000..dbc79d7d77bf8 --- /dev/null +++ b/modules/web-console/e2e/testenv/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/e2e/testenv/nginx/web-console.conf b/modules/web-console/e2e/testenv/nginx/web-console.conf new file mode 100644 index 0000000000000..c57c0d45dad10 --- /dev/null +++ b/modules/web-console/e2e/testenv/nginx/web-console.conf @@ -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. +# + +upstream backend-api { + server localhost:3000; +} + +server { + listen 9001; + 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 { + 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 /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/package.json b/modules/web-console/frontend/package.json index 7f7671a826923..c30f80c6aeb80 100644 --- a/modules/web-console/frontend/package.json +++ b/modules/web-console/frontend/package.json @@ -8,6 +8,8 @@ "dev": "npm start", "build": "webpack --config ./webpack/webpack.prod.babel.js", "test": "karma start ./test/karma.conf.js", + "enve2e": "node ./test/e2e/envtools.js start", + "teste2e": "node ./test/e2e/testcafe.js --env=true", "eslint": "eslint --format node_modules/eslint-friendly-formatter app/ controllers/ ignite_modules/ -- --eff-by-issue" }, "author": "", @@ -74,7 +76,6 @@ "file-loader": "0.11.2", "file-saver": "1.3.3", "font-awesome": "4.7.0", - "glob": "7.1.2", "html-loader": "0.4.5", "html-webpack-plugin": "2.29.0", "jquery": "3.2.1", @@ -104,6 +105,9 @@ }, "devDependencies": { "chai": "4.1.0", + "copy-dir": "^0.3.0", + "fs-extra": "^4.0.2", + "glob": "^7.1.2", "jasmine-core": "2.6.4", "karma": "1.7.0", "karma-babel-preprocessor": "6.0.1", @@ -113,10 +117,17 @@ "karma-phantomjs-launcher": "1.0.4", "karma-teamcity-reporter": "1.0.0", "karma-webpack": "2.0.4", + "minimist": "^1.2.0", "mocha": "3.4.2", "mocha-teamcity-reporter": "1.1.1", + "mongodb": "2.2.33", + "node-cmd": "3.0.0", + "objectid": "3.2.1", "phantomjs-prebuilt": "2.1.14", "sinon": "2.3.8", - "type-detect": "4.0.3" + "testcafe": "0.18.5", + "testcafe-angular-selectors": "0.3.0", + "type-detect": "4.0.3", + "util": "0.10.3" } } diff --git a/modules/web-console/frontend/public/stylesheets/style.scss b/modules/web-console/frontend/public/stylesheets/style.scss index baa49f87f7a14..d94f398c4ceff 100644 --- a/modules/web-console/frontend/public/stylesheets/style.scss +++ b/modules/web-console/frontend/public/stylesheets/style.scss @@ -1064,6 +1064,10 @@ button.form-control { .popover-content { padding: 5px; + + button { + margin-left: 5px; + } } .popover:focus { diff --git a/modules/web-console/frontend/views/settings/profile.tpl.pug b/modules/web-console/frontend/views/settings/profile.tpl.pug index 0dc35e40861c2..8f19b160f1151 100644 --- a/modules/web-console/frontend/views/settings/profile.tpl.pug +++ b/modules/web-console/frontend/views/settings/profile.tpl.pug @@ -60,7 +60,7 @@ mixin lbl-not-required(txt) a(ng-click='toggleToken()') {{expandedToken ? 'Cancel security token changing...' : 'Show security token...'}} div(ng-if='expandedToken') +lbl('Security token:') - label {{user.token || 'No security token. Regenerate please.'}} + label#current-security-token {{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.icon-help(ng-if=lines bs-tooltip='' data-title='The security token is used for authorization of web agent') diff --git a/modules/web-console/frontend/views/signin.tpl.pug b/modules/web-console/frontend/views/signin.tpl.pug index a053b86002f7a..29da8b15b4a6b 100644 --- a/modules/web-console/frontend/views/signin.tpl.pug +++ b/modules/web-console/frontend/views/signin.tpl.pug @@ -80,7 +80,7 @@ web-console-header | I agree to the #[a(ui-sref='{{::terms.termsState}}' target='_blank') terms and conditions] .col-xs-12.col-md-11 .login-footer(ng-show='action == "signup"') - a.labelField(ng-click='action = "password/forgot"' ignite-on-click-focus='signin_email') Forgot password? + a#password-forgot-signup.labelField(ng-click='action = "password/forgot"' ignite-on-click-focus='signin_email') Forgot password? a.labelLogin(ng-click='action = "signin"' ignite-on-click-focus='signin_email') Sign In button#signup.btn.btn-primary(ng-click='auth(action, ui)' ng-disabled='form.$invalid') Sign Up .col-xs-12.col-md-11 @@ -89,7 +89,7 @@ web-console-header button#forgot.btn.btn-primary(ng-click='forgotPassword(ui)' ng-disabled='form.$invalid') Send it to me .col-xs-12.col-md-11 .login-footer(ng-show='action == "signin"') - a.labelField(ng-click='action = "password/forgot"' ignite-on-click-focus='signin_email') Forgot password? + a#password-forgot-signin.labelField(ng-click='action = "password/forgot"' ignite-on-click-focus='signin_email') Forgot password? a.labelLogin(ng-click='action = "signup"' ignite-on-click-focus='first_name') Sign Up button#login.btn.btn-primary(ng-click='auth(action, ui)' ng-disabled='form.$invalid') Sign In diff --git a/modules/web-console/frontend/views/templates/validation-error.tpl.pug b/modules/web-console/frontend/views/templates/validation-error.tpl.pug index 13deb9b455409..2e0e42362c85b 100644 --- a/modules/web-console/frontend/views/templates/validation-error.tpl.pug +++ b/modules/web-console/frontend/views/templates/validation-error.tpl.pug @@ -20,6 +20,6 @@ table tr td - label {{content}}   + label#popover-validation-message {{content}} td button.close(id='popover-btn-close' ng-click='$hide()') × diff --git a/parent/pom.xml b/parent/pom.xml index ad6186d254960..f34559ddc2ad5 100644 --- a/parent/pom.xml +++ b/parent/pom.xml @@ -916,6 +916,7 @@ **/.dockerignore **/backend/config/settings.json.sample **/backend/node_modules/** + **/e2e/testcafe/node_modules/** **/frontend/build/** **/frontend/public/images/**/*.png **/frontend/public/images/**/*.svg From 06e891f1161af598e0aa4665f7a6047637d1c476 Mon Sep 17 00:00:00 2001 From: Dmitriy Shabalin Date: Thu, 25 Jan 2018 16:51:44 +0700 Subject: [PATCH 037/314] IGNITE-7522 Web Console: Fixed cluster selector state after cluster restart. (cherry picked from commit ac3cfb8) --- .../frontend/app/modules/agent/AgentManager.service.js | 3 +++ 1 file changed, 3 insertions(+) diff --git a/modules/web-console/frontend/app/modules/agent/AgentManager.service.js b/modules/web-console/frontend/app/modules/agent/AgentManager.service.js index 7bc23a98b49ec..0e1c1bf6bfdd2 100644 --- a/modules/web-console/frontend/app/modules/agent/AgentManager.service.js +++ b/modules/web-console/frontend/app/modules/agent/AgentManager.service.js @@ -52,6 +52,9 @@ class ConnectionState { this.clusters = clusters; + if (_.isEmpty(this.clusters)) + this.cluster = null; + if (_.isNil(this.cluster)) this.cluster = _.head(clusters); From 291cb2c88118ccffebcf3383db629647faec1eee Mon Sep 17 00:00:00 2001 From: Dmitriy Shabalin Date: Thu, 25 Jan 2018 17:33:13 +0700 Subject: [PATCH 038/314] IGNITE-7529 Web Console: Refactor UIGrid column filters. (cherry picked from commit 08658ea) --- .../app/components/grid-no-data/controller.js | 3 ++ .../components/ui-grid-filters/directive.js | 39 +++++++++++++++++-- .../app/components/ui-grid-filters/style.scss | 12 ++++++ .../components/ui-grid-filters/template.pug | 13 ++++++- 4 files changed, 61 insertions(+), 6 deletions(-) diff --git a/modules/web-console/frontend/app/components/grid-no-data/controller.js b/modules/web-console/frontend/app/components/grid-no-data/controller.js index 95e8a5e150948..2451e033962a9 100644 --- a/modules/web-console/frontend/app/components/grid-no-data/controller.js +++ b/modules/web-console/frontend/app/components/grid-no-data/controller.js @@ -30,6 +30,8 @@ export default class { $onChanges(changes) { if (changes && 'gridApi' in changes && changes.gridApi.currentValue) { + this.applyValues(); + this.gridApi.core.on.rowsVisibleChanged(this.$scope, () => { this.applyValues(); }); @@ -39,6 +41,7 @@ export default class { applyValues() { if (!this.gridApi.grid.rows.length) { this.noData = true; + return; } diff --git a/modules/web-console/frontend/app/components/ui-grid-filters/directive.js b/modules/web-console/frontend/app/components/ui-grid-filters/directive.js index 2e18933edd690..6602c11d9f074 100644 --- a/modules/web-console/frontend/app/components/ui-grid-filters/directive.js +++ b/modules/web-console/frontend/app/components/ui-grid-filters/directive.js @@ -22,15 +22,20 @@ export default function uiGridFilters(uiGridConstants) { return { require: 'uiGrid', link: { - pre(scope, el, attr, grid) { - if (!grid.grid.options.enableFiltering) return; - grid.grid.options.columnDefs.filter((cd) => cd.multiselectFilterOptions).forEach((cd) => { + pre(scope, el, attr, gridApi) { + if (!gridApi.grid.options.enableFiltering) return; + + const applyMultiselectFilter = (cd) => { cd.headerCellTemplate = template; + cd.filter = { type: uiGridConstants.filter.SELECT, term: cd.multiselectFilterOptions.map((t) => t.value), condition(searchTerm, cellValue) { - return searchTerm.includes(cellValue); + if (cellValue) + return Array.isArray(cellValue) ? _.intersection(searchTerm, cellValue).length : searchTerm.includes(cellValue); + + return true; }, selectOptions: cd.multiselectFilterOptions, $$selectOptionsMapping: cd.multiselectFilterOptions.reduce((a, v) => Object.assign(a, {[v.value]: v.label}), {}), @@ -53,6 +58,32 @@ export default function uiGridFilters(uiGridConstants) { `; } + }; + + const updateMultiselectOptionsHandler = (gridApi, colDef) => { + if (!gridApi) + return; + + const col = gridApi.grid.getColumn(colDef.name); + const selectOptions = colDef.multiselectFilterOptionsFn(gridApi.grid, col.filter); + + if (selectOptions.length === col.filter.selectOptions.length) + return; + + col.filter.term = selectOptions.map((t) => t.value); + col.filter.selectOptions = selectOptions; + }; + + gridApi.grid.options.columnDefs.filter((cd) => cd.multiselectFilterOptions).forEach(applyMultiselectFilter); + + gridApi.grid.options.columnDefs.filter((cd) => cd.multiselectFilterOptionsFn).forEach((cd) => { + cd.multiselectFilterOptions = cd.multiselectFilterOptions || []; + applyMultiselectFilter(cd); + + if (cd.multiselectFilterDialog) + cd.filter.selectDialog = cd.multiselectFilterDialog; + + gridApi.grid.api.core.on.rowsVisibleChanged(scope, (gridApi) => updateMultiselectOptionsHandler(gridApi, cd)); }); } } diff --git a/modules/web-console/frontend/app/components/ui-grid-filters/style.scss b/modules/web-console/frontend/app/components/ui-grid-filters/style.scss index 629cbadafd27e..a8d7a84d867de 100644 --- a/modules/web-console/frontend/app/components/ui-grid-filters/style.scss +++ b/modules/web-console/frontend/app/components/ui-grid-filters/style.scss @@ -17,6 +17,7 @@ .ui-grid-filters[role="columnheader"] { display: flex; + flex-wrap: nowrap !important; // Decrease horizontal padding because multiselect button already has it padding-left: 8px !important; @@ -26,11 +27,22 @@ flex: auto !important; } + .uigf-btn { + font-weight: normal; + + &--active { + font-weight: bold; + } + } + .ui-grid-cell-contents[role="button"] { flex: auto !important; flex-basis: 100% !important; padding: 0 !important; + margin-left: -10px; overflow: visible !important; + + font-weight: normal; } } diff --git a/modules/web-console/frontend/app/components/ui-grid-filters/template.pug b/modules/web-console/frontend/app/components/ui-grid-filters/template.pug index c898078cb2396..4cdcc8705f6a2 100644 --- a/modules/web-console/frontend/app/components/ui-grid-filters/template.pug +++ b/modules/web-console/frontend/app/components/ui-grid-filters/template.pug @@ -20,9 +20,9 @@ ng-class="{'ui-grid-filter-cancel-button-hidden' : colFilter.disableCancelFilterButton === true }" ng-switch='colFilter.type') div(ng-switch-when='select') - button.btn-ignite.btn-ignite--link-dashed-success( + button.btn-ignite.btn-ignite--link-dashed-success.uigf-btn( ng-class=`{ - 'bold': colFilter.term.length !== colFilter.selectOptions.length + 'uigf-btn--active': colFilter.term.length !== colFilter.selectOptions.length }` type='button' title='{{ colFilter.$$multiselectFilterTooltip() }}' @@ -34,6 +34,15 @@ data-placement='bottom-left' protect-from-bs-select-render ) {{ col.displayName }} + div(ng-switch-when='dialog') + button.btn-ignite.btn-ignite--link-dashed-success.uigf-btn( + ng-class=`{ + 'uigf-btn--active': colFilter.term.length !== colFilter.selectOptions.length + }` + ng-click='colFilter.selectDialog(grid, colFilter)' + type='button' + title='{{ colFilter.$$multiselectFilterTooltip() }}' + ) {{ col.displayName }} .ui-grid-cell-contents(role='button') button.btn-ignite.btn-ignite--link-dashed-success( From 443eafc718685e66c9a60058bd0ab56d88f9f0a6 Mon Sep 17 00:00:00 2001 From: alexdel Date: Thu, 25 Jan 2018 21:38:36 +0700 Subject: [PATCH 039/314] IGNITE-7064 Web Console: Minor test fix. (cherry picked from commit 4b6d9ad) --- modules/web-console/e2e/testcafe/fixtures/auth.js | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/web-console/e2e/testcafe/fixtures/auth.js b/modules/web-console/e2e/testcafe/fixtures/auth.js index d6d0226c9abdf..8f72b8310912e 100644 --- a/modules/web-console/e2e/testcafe/fixtures/auth.js +++ b/modules/web-console/e2e/testcafe/fixtures/auth.js @@ -142,7 +142,7 @@ test('Forbid Ignite signing up of already existing user', async(t) => { .click(Selector('span').withText('Brazil')) .click('#signup'); - await t.expect(Selector('#popover-validation-message').withText('A user with the given username is already registered.').exists).ok(); + await t.expect(Selector('#popover-validation-message').withText('A user with the given username is already registered').exists).ok(); }); From 6f1df5c40100363b5922734223a774ff1d6a008e Mon Sep 17 00:00:00 2001 From: Ilya Borisov Date: Fri, 26 Jan 2018 16:07:47 +0700 Subject: [PATCH 040/314] IGNITE-7031 Web Console: Refactored confirmation cancellation logic. (cherry picked from commit 92ae3fe) --- modules/web-console/frontend/app/app.js | 2 ++ .../input-dialog/input-dialog.service.js | 3 +- .../components/user-notifications/service.js | 3 +- .../frontend/app/errors/CancellationError.js | 28 +++++++++++++++++++ .../frontend/app/services/Confirm.service.js | 3 +- .../app/services/ConfirmBatch.service.js | 3 +- .../frontend/app/services/Messages.service.js | 4 ++- .../frontend/app/services/exceptionHandler.js | 27 ++++++++++++++++++ 8 files changed, 68 insertions(+), 5 deletions(-) create mode 100644 modules/web-console/frontend/app/errors/CancellationError.js create mode 100644 modules/web-console/frontend/app/services/exceptionHandler.js diff --git a/modules/web-console/frontend/app/app.js b/modules/web-console/frontend/app/app.js index d127a969a80b2..dcb369d80263f 100644 --- a/modules/web-console/frontend/app/app.js +++ b/modules/web-console/frontend/app/app.js @@ -93,6 +93,7 @@ import UnsavedChangesGuard from './services/UnsavedChangesGuard.service'; import Clusters from './services/Clusters'; import Caches from './services/Caches'; import {CSV} from './services/CSV'; +import {$exceptionHandler} from './services/exceptionHandler.js'; import AngularStrapTooltip from './services/AngularStrapTooltip.decorator'; import AngularStrapSelect from './services/AngularStrapSelect.decorator'; @@ -220,6 +221,7 @@ angular.module('ignite-console', [ // Ignite modules. IgniteModules.name ]) +.service($exceptionHandler.name, $exceptionHandler) // Directives. .directive(...igniteAutoFocus) .directive(...igniteBsAffixUpdate) diff --git a/modules/web-console/frontend/app/components/input-dialog/input-dialog.service.js b/modules/web-console/frontend/app/components/input-dialog/input-dialog.service.js index 4a48b464a03da..cfd61714df123 100644 --- a/modules/web-console/frontend/app/components/input-dialog/input-dialog.service.js +++ b/modules/web-console/frontend/app/components/input-dialog/input-dialog.service.js @@ -17,6 +17,7 @@ import controller from './input-dialog.controller'; import templateUrl from './input-dialog.tpl.pug'; +import {CancellationError} from 'app/errors/CancellationError'; export default class InputDialog { static $inject = ['$modal', '$q']; @@ -55,7 +56,7 @@ export default class InputDialog { const modalHide = modal.hide; - modal.hide = () => deferred.reject('cancelled'); + modal.hide = () => deferred.reject(new CancellationError()); return deferred.promise .finally(modalHide); diff --git a/modules/web-console/frontend/app/components/user-notifications/service.js b/modules/web-console/frontend/app/components/user-notifications/service.js index f47890251648a..2fc8064f132df 100644 --- a/modules/web-console/frontend/app/components/user-notifications/service.js +++ b/modules/web-console/frontend/app/components/user-notifications/service.js @@ -17,6 +17,7 @@ import controller from './controller'; import templateUrl from './template.tpl.pug'; +import {CancellationError} from 'app/errors/CancellationError'; export default class UserNotificationsService { static $inject = ['$http', '$modal', '$q', 'IgniteMessages']; @@ -49,7 +50,7 @@ export default class UserNotificationsService { const modalHide = modal.hide; - modal.hide = () => deferred.reject('cancelled'); + modal.hide = () => deferred.reject(new CancellationError()); return deferred.promise .finally(modalHide) diff --git a/modules/web-console/frontend/app/errors/CancellationError.js b/modules/web-console/frontend/app/errors/CancellationError.js new file mode 100644 index 0000000000000..17f0035f63396 --- /dev/null +++ b/modules/web-console/frontend/app/errors/CancellationError.js @@ -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. + */ + +export class CancellationError extends Error { + constructor(message = 'Cancelled by user') { + super(message); + + // Workaround for Babel issue with extend: https://github.com/babel/babel/issues/3083#issuecomment-315569824 + this.constructor = CancellationError; + + // eslint-disable-next-line no-proto + this.__proto__ = CancellationError.prototype; + } +} diff --git a/modules/web-console/frontend/app/services/Confirm.service.js b/modules/web-console/frontend/app/services/Confirm.service.js index 2429f4aad051b..6fe7ab8ac7764 100644 --- a/modules/web-console/frontend/app/services/Confirm.service.js +++ b/modules/web-console/frontend/app/services/Confirm.service.js @@ -16,6 +16,7 @@ */ import templateUrl from 'views/templates/confirm.tpl.pug'; +import {CancellationError} from 'app/errors/CancellationError'; // Confirm popup service. export default ['IgniteConfirm', ['$rootScope', '$q', '$modal', '$animate', ($root, $q, $modal, $animate) => { @@ -46,7 +47,7 @@ export default ['IgniteConfirm', ['$rootScope', '$q', '$modal', '$animate', ($ro scope.confirmCancel = () => { _hide(); - deferred.reject({cancelled: true}); + deferred.reject(new CancellationError()); }; /** diff --git a/modules/web-console/frontend/app/services/ConfirmBatch.service.js b/modules/web-console/frontend/app/services/ConfirmBatch.service.js index 8b473d80fc6f1..2739f294521bc 100644 --- a/modules/web-console/frontend/app/services/ConfirmBatch.service.js +++ b/modules/web-console/frontend/app/services/ConfirmBatch.service.js @@ -16,6 +16,7 @@ */ import templateUrl from 'views/templates/batch-confirm.tpl.pug'; +import {CancellationError} from 'app/errors/CancellationError'; // Service for confirm or skip several steps. export default ['IgniteConfirmBatch', ['$rootScope', '$q', '$modal', ($root, $q, $modal) => { @@ -33,7 +34,7 @@ export default ['IgniteConfirmBatch', ['$rootScope', '$q', '$modal', ($root, $q, scope.confirmModal.hide(); if (cancel) - scope.deferred.reject('cancelled'); + scope.deferred.reject(new CancellationError()); else scope.deferred.resolve(); }; diff --git a/modules/web-console/frontend/app/services/Messages.service.js b/modules/web-console/frontend/app/services/Messages.service.js index 6a6b9d316cc4f..39ffd3c38f415 100644 --- a/modules/web-console/frontend/app/services/Messages.service.js +++ b/modules/web-console/frontend/app/services/Messages.service.js @@ -15,6 +15,8 @@ * limitations under the License. */ +import {CancellationError} from 'app/errors/CancellationError'; + // Service to show various information and error messages. export default ['IgniteMessages', ['$alert', ($alert) => { // Common instance of alert modal. @@ -70,7 +72,7 @@ export default ['IgniteMessages', ['$alert', ($alert) => { errorMessage, hideAlert, showError(message, err) { - if (message && message.cancelled) + if (message instanceof CancellationError) return false; _showMessage(message, err, 'danger', 10); diff --git a/modules/web-console/frontend/app/services/exceptionHandler.js b/modules/web-console/frontend/app/services/exceptionHandler.js new file mode 100644 index 0000000000000..0d9cf3d9aa702 --- /dev/null +++ b/modules/web-console/frontend/app/services/exceptionHandler.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 {CancellationError} from 'app/errors/CancellationError'; + +export function $exceptionHandler($log) { + return function(exception, cause) { + if (exception instanceof CancellationError) return; + $log.error(exception, cause); + }; +} + +$exceptionHandler.$inject = ['$log']; From 16982825fb06ff2724ba4583781cc15443c4f46d Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Fri, 2 Feb 2018 17:07:57 +0700 Subject: [PATCH 041/314] IGNITE-7610 Web Console: Profile page refactored to component. (cherry picked from commit 958975e) --- modules/web-console/frontend/app/app.js | 7 +- .../page-profile/component.js} | 24 ++--- .../app/components/page-profile/controller.js | 79 ++++++++++++++++ .../app/components/page-profile/index.js | 36 +++++++ .../components/page-profile/template.pug} | 48 +++++----- .../frontend/app/data/countries.json | 4 + .../controllers/profile-controller.js | 94 ------------------- 7 files changed, 152 insertions(+), 140 deletions(-) rename modules/web-console/frontend/app/{modules/states/profile.state.js => components/page-profile/component.js} (64%) create mode 100644 modules/web-console/frontend/app/components/page-profile/controller.js create mode 100644 modules/web-console/frontend/app/components/page-profile/index.js rename modules/web-console/frontend/{views/settings/profile.tpl.pug => app/components/page-profile/template.pug} (60%) delete mode 100644 modules/web-console/frontend/controllers/profile-controller.js diff --git a/modules/web-console/frontend/app/app.js b/modules/web-console/frontend/app/app.js index dcb369d80263f..c19018c7020ca 100644 --- a/modules/web-console/frontend/app/app.js +++ b/modules/web-console/frontend/app/app.js @@ -29,7 +29,6 @@ import './modules/states/signin.state'; import './modules/states/logout.state'; import './modules/states/password.state'; import './modules/states/configuration.state'; -import './modules/states/profile.state'; import './modules/states/admin.state'; import './modules/states/errors.state'; @@ -108,7 +107,6 @@ import uiGridSubcategories from './filters/uiGridSubcategories.filter'; import id8 from './filters/id8.filter'; // Controllers -import profile from 'Controllers/profile-controller'; import resetPassword from './controllers/reset-password.controller'; // Components @@ -136,6 +134,8 @@ import listEditable from './components/list-editable'; import clusterSelector from './components/cluster-selector'; import connectedClusters from './components/connected-clusters'; +import pageProfile from './components/page-profile'; + import igniteServices from './services'; // Inject external modules. @@ -182,7 +182,6 @@ angular.module('ignite-console', [ 'ignite-console.states.logout', 'ignite-console.states.password', 'ignite-console.states.configuration', - 'ignite-console.states.profile', 'ignite-console.states.admin', 'ignite-console.states.errors', // Common modules. @@ -218,6 +217,7 @@ angular.module('ignite-console', [ clusterSelector.name, connectedClusters.name, igniteListOfRegisteredUsers.name, + pageProfile.name, // Ignite modules. IgniteModules.name ]) @@ -269,7 +269,6 @@ angular.module('ignite-console', [ .service(CSV.name, CSV) // Controllers. .controller(...resetPassword) -.controller(...profile) // Filters. .filter('byName', byName) .filter('defaultName', defaultName) diff --git a/modules/web-console/frontend/app/modules/states/profile.state.js b/modules/web-console/frontend/app/components/page-profile/component.js similarity index 64% rename from modules/web-console/frontend/app/modules/states/profile.state.js rename to modules/web-console/frontend/app/components/page-profile/component.js index 35e0ae65cea3e..9578339e26501 100644 --- a/modules/web-console/frontend/app/modules/states/profile.state.js +++ b/modules/web-console/frontend/app/components/page-profile/component.js @@ -15,22 +15,10 @@ * limitations under the License. */ -import angular from 'angular'; +import template from './template.pug'; +import controller from './controller'; -import templateUrl from 'views/settings/profile.tpl.pug'; - -angular -.module('ignite-console.states.profile', [ - 'ui.router' -]) -.config(['$stateProvider', function($stateProvider) { - // set up the states - $stateProvider.state('base.settings.profile', { - url: '/profile', - templateUrl, - permission: 'profile', - tfMetaTags: { - title: 'User profile' - } - }); -}]); +export default { + template, + controller +}; diff --git a/modules/web-console/frontend/app/components/page-profile/controller.js b/modules/web-console/frontend/app/components/page-profile/controller.js new file mode 100644 index 0000000000000..b60340b74d757 --- /dev/null +++ b/modules/web-console/frontend/app/components/page-profile/controller.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. + */ + +export default class PageProfileController { + static $inject = [ + '$rootScope', '$scope', '$http', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteFocus', 'IgniteConfirm', 'IgniteCountries', 'User' + ]; + + constructor($root, $scope, $http, LegacyUtils, Messages, Focus, Confirm, Countries, User) { + Object.assign(this, {$root, $scope, $http, LegacyUtils, Messages, Focus, Confirm, Countries, User}); + } + + $onInit() { + const self = this; + + self.ui = {}; + + this.User.read() + .then((user) => self.ui.user = angular.copy(user)); + + self.ui.countries = this.Countries.getAll(); + } + + toggleToken() { + this.ui.expandedToken = !this.ui.expandedToken; + + if (!this.ui.expandedToken) + this.ui.user.token = this.$root.user.token; + } + + generateToken() { + this.Confirm.confirm('Are you sure you want to change security token?') + .then(() => this.ui.user.token = this.LegacyUtils.randomString(20)); + } + + togglePassword() { + this.ui.expandedPassword = !this.ui.expandedPassword; + + if (this.ui.expandedPassword) + this.Focus.move('profile_password'); + else { + delete this.ui.user.password; + delete this.ui.user.confirm; + } + } + + saveUser() { + return this.$http.post('/api/v1/profile/save', this.ui.user) + .then(this.User.load) + .then(() => { + if (this.ui.expandedPassword) + this.togglePassword(); + + if (this.ui.expandedToken) + this.toggleToken(); + + this.Messages.showInfo('Profile saved.'); + + this.Focus.move('profile-username'); + + this.$root.$broadcast('user', this.ui.user); + }) + .catch((res) => this.Messages.showError('Failed to save profile: ', res)); + } +} diff --git a/modules/web-console/frontend/app/components/page-profile/index.js b/modules/web-console/frontend/app/components/page-profile/index.js new file mode 100644 index 0000000000000..d9921b3083836 --- /dev/null +++ b/modules/web-console/frontend/app/components/page-profile/index.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. + */ + +import angular from 'angular'; +import component from './component'; + +export default angular + .module('ignite-console.page-profile', [ + 'ignite-console.user' + ]) + .config(['$stateProvider', ($stateProvider) => { + // set up the states + $stateProvider.state('base.settings.profile', { + url: '/profile', + component: 'pageProfile', + permission: 'profile', + tfMetaTags: { + title: 'User profile' + } + }); + }]) + .component('pageProfile', component); diff --git a/modules/web-console/frontend/views/settings/profile.tpl.pug b/modules/web-console/frontend/app/components/page-profile/template.pug similarity index 60% rename from modules/web-console/frontend/views/settings/profile.tpl.pug rename to modules/web-console/frontend/app/components/page-profile/template.pug index 8f19b160f1151..5331670ddaf98 100644 --- a/modules/web-console/frontend/views/settings/profile.tpl.pug +++ b/modules/web-console/frontend/app/components/page-profile/template.pug @@ -20,64 +20,64 @@ mixin lbl(txt) mixin lbl-not-required(txt) label.col-sm-2.labelFormField #{txt} -.row(ng-controller='profileController') +.row .docs-content .docs-header h1 User profile hr .docs-body - form.form-horizontal(name='profileForm' novalidate) + form.form-horizontal(name='$ctrl.form' novalidate) .col-sm-10(style='padding: 0') .details-row +lbl('First name:') .col-xs-5.col-sm-4 - input#profile-firstname.form-control(ignite-on-enter-focus-move='profile-lastname' type='text' ng-model='user.firstName' placeholder='Input first name' required ignite-auto-focus) + input#profile-firstname.form-control(ignite-on-enter-focus-move='profile-lastname' type='text' ng-model='$ctrl.ui.user.firstName' placeholder='Input first name' required ignite-auto-focus) .details-row +lbl('Last name:') .col-xs-5.col-sm-4 - input#profile-lastname.form-control(ignite-on-enter-focus-move='profile-email' type='text' ng-model='user.lastName' placeholder='Input last name' required) + input#profile-lastname.form-control(ignite-on-enter-focus-move='profile-email' type='text' ng-model='$ctrl.ui.user.lastName' placeholder='Input last name' required) .details-row +lbl('Email:') .col-xs-5.col-sm-4 - input#profile-email.form-control(ignite-on-enter-focus-move='profile-phone' type='email' ng-model='user.email' placeholder='Input email' required) + input#profile-email.form-control(ignite-on-enter-focus-move='profile-company' type='email' ng-model='$ctrl.ui.user.email' placeholder='Input email' required) .details-row +lbl-not-required('Phone:') .col-xs-5.col-sm-4 - input#profile-phone.form-control(ignite-on-enter-focus-move='profile-company' type='tel' ng-model='user.phone' placeholder='Input phone') + input#profile-phone.form-control(ignite-on-enter-focus-move='profile-company' type='tel' ng-model='$ctrl.ui.user.phone' placeholder='Input phone') .details-row +lbl('Company:') .col-xs-5.col-sm-4 - input#profile-company.form-control(ignite-on-enter-focus-move='profile-country' type='text' ng-model='user.company' placeholder='Input company name' required) + input#profile-company.form-control(ignite-on-enter-focus-move='profile-country' type='text' ng-model='$ctrl.ui.user.company' placeholder='Input company name' required) .details-row +lbl('Country:') .col-xs-5.col-sm-4 - button#profile-country.select-toggle.form-control(bs-select bs-options='item.name as item.name for item in countries' type='text' ng-model='user.country' placeholder='Choose your country' ng-required='true') - .details-row + button#profile-country.select-toggle.form-control(bs-select bs-options='item.name as item.name for item in $ctrl.ui.countries' type='text' ng-model='$ctrl.ui.user.country' placeholder='Choose your country' ng-required='true') + .details-row#security-token-section .advanced-options i.fa( - ng-click='toggleToken()' - ng-class='expandedToken ? "fa-chevron-circle-down" : "fa-chevron-circle-right"') - a(ng-click='toggleToken()') {{expandedToken ? 'Cancel security token changing...' : 'Show security token...'}} - div(ng-if='expandedToken') + ng-click='$ctrl.toggleToken()' + ng-class='$ctrl.ui.expandedToken ? "fa-chevron-circle-down" : "fa-chevron-circle-right"') + a(ng-click='$ctrl.toggleToken()') {{$ctrl.ui.expandedToken ? 'Cancel security token changing...' : 'Show security token...'}} + div(ng-if='$ctrl.ui.expandedToken') +lbl('Security token:') - label#current-security-token {{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') + label#current-security-token {{$ctrl.ui.user.token || 'No security token. Regenerate please.'}} + i.tipLabel.fa.fa-refresh(ng-click='$ctrl.generateToken()' bs-tooltip='' data-title='Generate random security token') + i.tipLabel.fa.fa-clipboard(ignite-copy-to-clipboard='{{$ctrl.ui.user.token}}' bs-tooltip='' data-title='Copy security token to clipboard') i.tipLabel.icon-help(ng-if=lines bs-tooltip='' data-title='The security token is used for authorization of web agent') - .details-row + .details-row#change-password-section .advanced-options i.fa( - ng-click='togglePassword()' - ng-class='expandedPassword ? "fa-chevron-circle-down" : "fa-chevron-circle-right"') - a(ng-click='togglePassword()') {{expandedPassword ? 'Cancel password changing...' : 'Change password...'}} - div(ng-if='expandedPassword') + ng-click='$ctrl.togglePassword()' + ng-class='$ctrl.ui.expandedPassword ? "fa-chevron-circle-down" : "fa-chevron-circle-right"') + a(ng-click='$ctrl.togglePassword()') {{$ctrl.ui.expandedPassword ? 'Cancel password changing...' : 'Change password...'}} + div(ng-if='$ctrl.ui.expandedPassword') .details-row +lbl('New password:') .col-xs-5.col-sm-4 - input#profile_password.form-control(ignite-on-enter-focus-move='profile_confirm' type='password' ng-model='user.password' placeholder='New password') + input#profile_password.form-control(ignite-on-enter-focus-move='profile_confirm' type='password' ng-model='$ctrl.ui.user.password' placeholder='New password' required) .details-row +lbl('Confirm:') .col-xs-5.col-sm-4 - input#profile_confirm.form-control(type='password' ng-model='user.confirm' ignite-match='user.password' placeholder='Confirm new password') + input#profile_confirm.form-control(type='password' ng-model='user.confirm' ignite-match='$ctrl.ui.user.password' placeholder='Confirm new password' required) .col-xs-12.col-sm-12.details-row - a.btn.btn-primary(ng-disabled='!profileCouldBeSaved()' ng-click='profileCouldBeSaved() && saveUser()' bs-tooltip='' data-title='{{saveBtnTipText()}}' data-placement='bottom' data-trigger='hover') Save + button.btn.btn-primary(ng-click='$ctrl.saveUser()' ng-disabled='$ctrl.form.$invalid') Save diff --git a/modules/web-console/frontend/app/data/countries.json b/modules/web-console/frontend/app/data/countries.json index f420f48524fe8..18dbe64869617 100644 --- a/modules/web-console/frontend/app/data/countries.json +++ b/modules/web-console/frontend/app/data/countries.json @@ -19,6 +19,10 @@ "name": "France", "code": "FRA" }, + { + "name": "Belgium", + "code": "BEL" + }, { "name": "Switzerland", "code": "CHE" diff --git a/modules/web-console/frontend/controllers/profile-controller.js b/modules/web-console/frontend/controllers/profile-controller.js deleted file mode 100644 index 87a880596c4bd..0000000000000 --- a/modules/web-console/frontend/controllers/profile-controller.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. - */ - -// Controller for Profile screen. -export default ['profileController', [ - '$rootScope', '$scope', '$http', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteFocus', 'IgniteConfirm', 'IgniteCountries', 'User', - function($root, $scope, $http, LegacyUtils, Messages, Focus, Confirm, Countries, User) { - User.read() - .then((user) => $scope.user = angular.copy(user)); - - $scope.countries = Countries.getAll(); - - $scope.generateToken = () => { - Confirm.confirm('Are you sure you want to change security token?') - .then(() => $scope.user.token = LegacyUtils.randomString(20)); - }; - - const _passwordValid = () => { - const cur = $scope.user; - - return !$scope.expandedPassword || (cur.password && cur.confirm && cur.password === cur.confirm); - }; - - const _profileChanged = () => { - const old = $root.user; - const cur = $scope.user; - - return !_.isEqual(old, cur); - }; - - $scope.toggleToken = () => { - $scope.expandedToken = !$scope.expandedToken; - - if (!$scope.expandedToken) - $scope.user.token = $root.user.token; - }; - - $scope.togglePassword = () => { - $scope.expandedPassword = !$scope.expandedPassword; - - if ($scope.expandedPassword) - Focus.move('profile_password'); - else { - delete $scope.user.password; - delete $scope.user.confirm; - } - }; - - $scope.profileCouldBeSaved = () => _profileChanged() && $scope.profileForm && $scope.profileForm.$valid && _passwordValid(); - - $scope.saveBtnTipText = () => { - if (!_profileChanged()) - return 'Nothing to save'; - - if (!_passwordValid()) - return 'Invalid password'; - - return $scope.profileForm && $scope.profileForm.$valid ? 'Save profile' : 'Invalid profile settings'; - }; - - $scope.saveUser = () => { - $http.post('/api/v1/profile/save', $scope.user) - .then(User.load) - .then(() => { - if ($scope.expandedPassword) - $scope.togglePassword(); - - if ($scope.expandedToken) - $scope.toggleToken(); - - Messages.showInfo('Profile saved.'); - - Focus.move('profile-username'); - - $root.$broadcast('user', $scope.user); - }) - .catch((res) => Messages.showError('Failed to save profile: ', res)); - }; - } -]]; From 9c6a52250e058c4546aef0d0ecee977c07076a1a Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Fri, 2 Feb 2018 17:09:37 +0700 Subject: [PATCH 042/314] IGNITE-7612 Web Console: Refactored mongoose schemas to separate module. (cherry picked from commit 71db707) --- modules/web-console/backend/app/mongo.js | 1127 +--------------- modules/web-console/backend/app/schemas.js | 1147 +++++++++++++++++ modules/web-console/backend/services/users.js | 1 + 3 files changed, 1159 insertions(+), 1116 deletions(-) create mode 100644 modules/web-console/backend/app/schemas.js diff --git a/modules/web-console/backend/app/mongo.js b/modules/web-console/backend/app/mongo.js index bd03d6ab2a68e..661d6f9edb0a9 100644 --- a/modules/web-console/backend/app/mongo.js +++ b/modules/web-console/backend/app/mongo.js @@ -17,7 +17,7 @@ 'use strict'; -const passportMongo = require('passport-local-mongoose'); +const _ = require('lodash'); // Fire me up! @@ -26,1130 +26,25 @@ const passportMongo = require('passport-local-mongoose'); */ module.exports = { implements: 'mongo', - inject: ['settings', 'mongoose'] + inject: ['settings', 'mongoose', 'schemas'] }; -const defineSchema = (mongoose) => { - const Schema = mongoose.Schema; +const defineSchema = (mongoose, schemas) => { const ObjectId = mongoose.Schema.Types.ObjectId; const result = { connection: mongoose.connection }; result.ObjectId = ObjectId; - // Define Account schema. - const AccountSchema = new Schema({ - firstName: String, - lastName: String, - email: String, - phone: String, - company: String, - country: String, - registered: Date, - lastLogin: Date, - lastActivity: Date, - admin: Boolean, - token: String, - resetPasswordToken: String - }); - - // Install passport plugin. - AccountSchema.plugin(passportMongo, { - usernameField: 'email', limitAttempts: true, lastLoginField: 'lastLogin', - usernameLowerCase: true - }); - - const transform = (doc, ret) => { - return { - _id: ret._id, - email: ret.email, - phone: ret.phone, - firstName: ret.firstName, - lastName: ret.lastName, - company: ret.company, - country: ret.country, - admin: ret.admin, - token: ret.token, - registered: ret.registered, - lastLogin: ret.lastLogin, - lastActivity: ret.lastActivity - }; - }; - - // Configure transformation to JSON. - AccountSchema.set('toJSON', { transform }); - - // Configure transformation to JSON. - AccountSchema.set('toObject', { transform }); - result.errCodes = { DUPLICATE_KEY_ERROR: 11000, DUPLICATE_KEY_UPDATE_ERROR: 11001 }; - // Define Account model. - result.Account = mongoose.model('Account', AccountSchema); - - // Define Space model. - result.Space = mongoose.model('Space', new Schema({ - name: String, - owner: {type: ObjectId, ref: 'Account'}, - demo: {type: Boolean, default: false}, - usedBy: [{ - permission: {type: String, enum: ['VIEW', 'FULL']}, - account: {type: ObjectId, ref: 'Account'} - }] - })); - - // Define Domain model schema. - const DomainModelSchema = new Schema({ - space: {type: ObjectId, ref: 'Space', index: true, required: true}, - caches: [{type: ObjectId, ref: 'Cache'}], - queryMetadata: {type: String, enum: ['Annotations', 'Configuration']}, - kind: {type: String, enum: ['query', 'store', 'both']}, - tableName: String, - keyFieldName: String, - valueFieldName: String, - databaseSchema: String, - databaseTable: String, - keyType: String, - valueType: {type: String}, - keyFields: [{ - databaseFieldName: String, - databaseFieldType: String, - javaFieldName: String, - javaFieldType: String - }], - valueFields: [{ - databaseFieldName: String, - databaseFieldType: String, - javaFieldName: String, - javaFieldType: String - }], - queryKeyFields: [String], - fields: [{name: String, className: String}], - aliases: [{field: String, alias: String}], - indexes: [{ - name: String, - indexType: {type: String, enum: ['SORTED', 'FULLTEXT', 'GEOSPATIAL']}, - fields: [{name: String, direction: Boolean}] - }], - generatePojo: 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, required: true}, - name: {type: String}, - groupName: {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']}, - partitionLossPolicy: {type: String, enum: ['READ_ONLY_SAFE', 'READ_ONLY_ALL', 'READ_WRITE_SAFE', 'READ_WRITE_ALL', 'IGNORE']}, - - affinity: { - kind: {type: String, enum: ['Default', 'Rendezvous', 'Fair', 'Custom']}, - Rendezvous: { - affinityBackupFilter: String, - partitions: Number, - excludeNeighbors: Boolean - }, - Fair: { - affinityBackupFilter: String, - partitions: Number, - excludeNeighbors: Boolean - }, - Custom: { - className: String - } - }, - - affinityMapper: String, - - nodeFilter: { - kind: {type: String, enum: ['Default', 'Exclude', 'IGFS', 'OnNodes', 'Custom']}, - Exclude: { - nodeId: String - }, - IGFS: { - igfs: {type: ObjectId, ref: 'Igfs'} - }, - Custom: { - className: String - } - }, - - backups: Number, - memoryMode: {type: String, enum: ['ONHEAP_TIERED', 'OFFHEAP_TIERED', 'OFFHEAP_VALUES']}, - offHeapMaxMemory: Number, - startSize: Number, - swapEnabled: Boolean, - - onheapCacheEnabled: Boolean, - - evictionPolicy: { - 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 - } - }, - - rebalanceMode: {type: String, enum: ['SYNC', 'ASYNC', 'NONE']}, - rebalanceBatchSize: Number, - rebalanceBatchesPrefetchCount: Number, - rebalanceOrder: Number, - rebalanceDelay: Number, - rebalanceTimeout: Number, - rebalanceThrottle: Number, - - cacheStoreFactory: { - kind: { - type: String, - enum: ['CacheJdbcPojoStoreFactory', 'CacheJdbcBlobStoreFactory', 'CacheHibernateBlobStoreFactory'] - }, - CacheJdbcPojoStoreFactory: { - dataSourceBean: String, - dialect: { - 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: { - connectVia: {type: String, enum: ['URL', 'DataSource']}, - connectionUrl: String, - user: String, - dataSourceBean: String, - dialect: { - type: String, - enum: ['Generic', 'Oracle', 'DB2', 'SQLServer', 'MySQL', 'PostgreSQL', 'H2'] - }, - initSchema: Boolean, - createTableQuery: String, - loadQuery: String, - insertQuery: String, - updateQuery: String, - deleteQuery: String - }, - CacheHibernateBlobStoreFactory: { - hibernateProperties: [{name: String, value: String}] - } - }, - storeKeepBinary: Boolean, - loadPreviousValue: Boolean, - readThrough: Boolean, - writeThrough: Boolean, - - writeBehindEnabled: Boolean, - writeBehindBatchSize: Number, - writeBehindFlushSize: Number, - writeBehindFlushFrequency: Number, - writeBehindFlushThreadCount: Number, - writeBehindCoalescing: {type: Boolean, default: true}, - - invalidate: Boolean, - defaultLockTimeout: Number, - atomicWriteOrderMode: {type: String, enum: ['CLOCK', 'PRIMARY']}, - writeSynchronizationMode: {type: String, enum: ['FULL_SYNC', 'FULL_ASYNC', 'PRIMARY_SYNC']}, - - sqlEscapeAll: Boolean, - sqlSchema: String, - sqlOnheapRowCacheSize: Number, - longQueryWarningTimeout: Number, - sqlFunctionClasses: [String], - snapshotableIndex: Boolean, - queryDetailMetricsSize: Number, - queryParallelism: Number, - statisticsEnabled: Boolean, - managementEnabled: Boolean, - readFromBackup: Boolean, - copyOnRead: Boolean, - maxConcurrentAsyncOperations: Number, - nearConfiguration: { - 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 - } - } - }, - 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 - } - } - }, - evictionFilter: String, - memoryPolicyName: String, - dataRegionName: String, - sqlIndexMaxInlineSize: Number, - topologyValidator: String - }); - - 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, required: true}, - name: {type: String}, - clusters: [{type: ObjectId, ref: 'Cluster'}], - affinnityGroupSize: Number, - blockSize: Number, - streamBufferSize: Number, - dataCacheName: String, - metaCacheName: String, - defaultMode: {type: String, enum: ['PRIMARY', 'PROXY', 'DUAL_SYNC', 'DUAL_ASYNC']}, - dualModeMaxPendingPutsSize: Number, - dualModePutExecutorService: String, - dualModePutExecutorServiceShutdown: Boolean, - fragmentizerConcurrentFiles: Number, - fragmentizerEnabled: Boolean, - fragmentizerThrottlingBlockLength: Number, - fragmentizerThrottlingDelay: Number, - ipcEndpointConfiguration: { - type: {type: String, enum: ['SHMEM', 'TCP']}, - host: String, - port: Number, - memorySize: Number, - tokenDirectoryPath: String, - threadCount: Number - }, - ipcEndpointEnabled: Boolean, - maxSpaceSize: Number, - maximumTaskRangeLength: Number, - managementPort: Number, - pathModes: [{path: String, mode: {type: String, enum: ['PRIMARY', 'PROXY', 'DUAL_SYNC', 'DUAL_ASYNC']}}], - perNodeBatchSize: Number, - perNodeParallelBatchCount: Number, - prefetchBlocks: Number, - sequentialReadsBeforePrefetch: Number, - trashPurgeTimeout: Number, - secondaryFileSystemEnabled: Boolean, - secondaryFileSystem: { - uri: String, - cfgPath: String, - userName: String - }, - colocateMetadata: Boolean, - relaxedConsistency: Boolean, - updateFileLengthOnFlush: Boolean + // Define models. + _.forEach(schemas, (schema, name) => { + result[name] = mongoose.model(name, schema); }); - 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, required: true}, - name: {type: String}, - activeOnStart: {type: Boolean, default: true}, - localHost: String, - discovery: { - localAddress: String, - localPort: Number, - localPortRange: Number, - addressResolver: String, - socketTimeout: Number, - ackTimeout: Number, - maxAckTimeout: Number, - networkTimeout: Number, - joinTimeout: Number, - threadPriority: Number, - heartbeatFrequency: Number, - maxMissedHeartbeats: Number, - maxMissedClientHeartbeats: Number, - topHistorySize: Number, - listener: String, - dataExchange: String, - metricsProvider: String, - reconnectCount: Number, - statisticsPrintFrequency: Number, - ipFinderCleanFrequency: Number, - authenticator: String, - forceServerMode: Boolean, - clientReconnectDisabled: Boolean, - kind: {type: String, enum: ['Vm', 'Multicast', 'S3', 'Cloud', 'GoogleStorage', 'Jdbc', 'SharedFs', 'ZooKeeper', 'Kubernetes']}, - Vm: { - addresses: [String] - }, - Multicast: { - multicastGroup: String, - multicastPort: Number, - responseWaitTime: Number, - addressRequestAttempts: Number, - localAddress: String, - addresses: [String] - }, - S3: { - 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, - credentialPath: String, - identity: String, - provider: String, - regions: [String], - zones: [String] - }, - GoogleStorage: { - projectName: String, - bucketName: String, - serviceAccountP12FilePath: String, - serviceAccountId: String, - addrReqAttempts: String - }, - Jdbc: { - initSchema: Boolean, - dataSourceBean: String, - dialect: { - type: String, - enum: ['Generic', 'Oracle', 'DB2', 'SQLServer', 'MySQL', 'PostgreSQL', 'H2'] - } - }, - SharedFs: { - path: String - }, - ZooKeeper: { - curator: String, - zkConnectionString: String, - retryPolicy: { - kind: {type: String, enum: ['ExponentialBackoff', 'BoundedExponentialBackoff', 'UntilElapsed', - 'NTimes', 'OneTime', 'Forever', 'Custom']}, - ExponentialBackoff: { - baseSleepTimeMs: Number, - maxRetries: Number, - maxSleepMs: Number - }, - BoundedExponentialBackoff: { - baseSleepTimeMs: Number, - maxSleepTimeMs: Number, - maxRetries: Number - }, - UntilElapsed: { - maxElapsedTimeMs: Number, - sleepMsBetweenRetries: Number - }, - NTimes: { - n: Number, - sleepMsBetweenRetries: Number - }, - OneTime: { - sleepMsBetweenRetry: Number - }, - Forever: { - retryIntervalMs: Number - }, - Custom: { - className: String - } - }, - basePath: String, - serviceName: String, - allowDuplicateRegistrations: Boolean - }, - Kubernetes: { - serviceName: String, - namespace: String, - masterUrl: String, - accountToken: String - } - }, - atomicConfiguration: { - backups: Number, - cacheMode: {type: String, enum: ['LOCAL', 'REPLICATED', 'PARTITIONED']}, - atomicSequenceReserveSize: Number, - affinity: { - kind: {type: String, enum: ['Default', 'Rendezvous', 'Custom']}, - Rendezvous: { - affinityBackupFilter: String, - partitions: Number, - excludeNeighbors: Boolean - }, - Custom: { - className: String - } - } - }, - binaryConfiguration: { - idMapper: String, - nameMapper: String, - serializer: String, - typeConfigurations: [{ - typeName: String, - idMapper: String, - nameMapper: String, - serializer: String, - enum: Boolean - }], - compactFooter: Boolean - }, - caches: [{type: ObjectId, ref: 'Cache'}], - clockSyncSamples: Number, - clockSyncFrequency: Number, - deploymentMode: {type: String, enum: ['PRIVATE', 'ISOLATED', 'SHARED', 'CONTINUOUS']}, - discoveryStartupDelay: Number, - 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']}, - OptimizedMarshaller: { - poolSize: Number, - requireSerializable: Boolean - } - }, - marshalLocalJobs: Boolean, - marshallerCacheKeepAliveTime: Number, - marshallerCacheThreadPoolSize: Number, - metricsExpireTime: Number, - metricsHistorySize: Number, - metricsLogFrequency: Number, - metricsUpdateFrequency: Number, - networkTimeout: Number, - networkSendRetryDelay: Number, - networkSendRetryCount: Number, - communication: { - listener: String, - localAddress: String, - localPort: Number, - localPortRange: Number, - sharedMemoryPort: Number, - directBuffer: Boolean, - directSendBuffer: Boolean, - idleConnectionTimeout: Number, - connectTimeout: Number, - maxConnectTimeout: Number, - reconnectCount: Number, - socketSendBuffer: Number, - socketReceiveBuffer: Number, - messageQueueLimit: Number, - slowClientQueueLimit: Number, - tcpNoDelay: Boolean, - ackSendThreshold: Number, - unacknowledgedMessagesBufferSize: Number, - socketWriteTimeout: Number, - selectorsCount: Number, - addressResolver: String - }, - connector: { - enabled: Boolean, - jettyPath: String, - host: String, - port: Number, - portRange: Number, - idleTimeout: Number, - idleQueryCursorTimeout: Number, - idleQueryCursorCheckFrequency: Number, - receiveBufferSize: Number, - sendBufferSize: Number, - sendQueueLimit: Number, - directBuffer: Boolean, - noDelay: Boolean, - selectorCount: Number, - threadPoolSize: Number, - messageInterceptor: String, - secretKey: String, - sslEnabled: Boolean, - sslClientAuth: Boolean, - sslFactory: String - }, - peerClassLoadingEnabled: Boolean, - peerClassLoadingLocalClassPathExclude: [String], - peerClassLoadingMissedResourcesCacheSize: Number, - peerClassLoadingThreadPoolSize: Number, - publicThreadPoolSize: Number, - swapSpaceSpi: { - kind: {type: String, enum: ['FileSwapSpaceSpi']}, - FileSwapSpaceSpi: { - baseDirectory: String, - readStripesNumber: Number, - maximumSparsity: Number, - maxWriteQueueSize: Number, - writeBufferSize: Number - } - }, - systemThreadPoolSize: Number, - timeServerPortBase: Number, - timeServerPortRange: Number, - transactionConfiguration: { - defaultTxConcurrency: {type: String, enum: ['OPTIMISTIC', 'PESSIMISTIC']}, - defaultTxIsolation: {type: String, enum: ['READ_COMMITTED', 'REPEATABLE_READ', 'SERIALIZABLE']}, - defaultTxTimeout: Number, - pessimisticTxLogLinger: Number, - pessimisticTxLogSize: Number, - txSerializableEnabled: Boolean, - txManagerFactory: String - }, - sslEnabled: Boolean, - sslContextFactory: { - keyAlgorithm: String, - keyStoreFilePath: String, - keyStoreType: String, - protocol: String, - trustStoreFilePath: String, - trustStoreType: String, - trustManagers: [String] - }, - rebalanceThreadPoolSize: Number, - odbc: { - odbcEnabled: Boolean, - endpointAddress: String, - socketSendBufferSize: Number, - socketReceiveBufferSize: Number, - maxOpenCursors: Number, - threadPoolSize: Number - }, - attributes: [{name: String, value: String}], - collision: { - kind: {type: String, enum: ['Noop', 'PriorityQueue', 'FifoQueue', 'JobStealing', 'Custom']}, - PriorityQueue: { - parallelJobsNumber: Number, - waitingJobsNumber: Number, - priorityAttributeKey: String, - jobPriorityAttributeKey: String, - defaultPriority: Number, - starvationIncrement: Number, - starvationPreventionEnabled: Boolean - }, - FifoQueue: { - parallelJobsNumber: Number, - waitingJobsNumber: Number - }, - JobStealing: { - activeJobsThreshold: Number, - waitJobsThreshold: Number, - messageExpireTime: Number, - maximumStealingAttempts: Number, - stealingEnabled: Boolean, - stealingAttributes: [{name: String, value: String}], - externalCollisionListener: String - }, - Custom: { - class: String - } - }, - failoverSpi: [{ - kind: {type: String, enum: ['JobStealing', 'Never', 'Always', 'Custom']}, - JobStealing: { - maximumFailoverAttempts: Number - }, - Always: { - maximumFailoverAttempts: Number - }, - Custom: { - class: String - } - }], - logger: { - kind: {type: 'String', enum: ['Log4j2', 'Null', 'Java', 'JCL', 'SLF4J', 'Log4j', 'Custom']}, - Log4j2: { - level: {type: String, enum: ['OFF', 'FATAL', 'ERROR', 'WARN', 'INFO', 'DEBUG', 'TRACE', 'ALL']}, - path: String - }, - Log4j: { - mode: {type: String, enum: ['Default', 'Path']}, - level: {type: String, enum: ['OFF', 'FATAL', 'ERROR', 'WARN', 'INFO', 'DEBUG', 'TRACE', 'ALL']}, - path: String - }, - Custom: { - class: String - } - }, - 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 - } - }], - clientConnectorConfiguration: { - enabled: Boolean, - host: String, - port: Number, - portRange: Number, - socketSendBufferSize: Number, - socketReceiveBufferSize: Number, - tcpNoDelay: {type: Boolean, default: true}, - maxOpenCursorsPerConnection: Number, - threadPoolSize: Number - }, - 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 - } - }], - 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 - } - }, - warmupClosure: String, - hadoopConfiguration: { - mapReducePlanner: { - kind: {type: String, enum: ['Weighted', 'Custom']}, - Weighted: { - localMapperWeight: Number, - remoteMapperWeight: Number, - localReducerWeight: Number, - remoteReducerWeight: Number, - preferLocalReducerThresholdWeight: Number - }, - Custom: { - className: String - } - }, - finishedJobInfoTtl: Number, - maxParallelTasks: Number, - maxTaskQueueSize: Number, - nativeLibraryNames: [String] - }, - serviceConfigurations: [{ - name: String, - service: String, - maxPerNodeCount: Number, - totalCount: Number, - nodeFilter: { - kind: {type: String, enum: ['Default', 'Exclude', 'IGFS', 'OnNodes', 'Custom']}, - Exclude: { - nodeId: String - }, - IGFS: { - igfs: {type: ObjectId, ref: 'Igfs'} - }, - Custom: { - className: String - } - }, - cache: {type: ObjectId, ref: 'Cache'}, - affinityKey: String - }], - cacheSanityCheckEnabled: {type: Boolean, default: true}, - classLoader: String, - consistentId: String, - failureDetectionTimeout: Number, - clientFailureDetectionTimeout: Number, - workDirectory: String, - lateAffinityAssignment: Boolean, - utilityCacheKeepAliveTime: Number, - asyncCallbackPoolSize: Number, - dataStreamerThreadPoolSize: Number, - queryThreadPoolSize: Number, - stripedPoolSize: Number, - serviceThreadPoolSize: Number, - utilityCacheThreadPoolSize: Number, - executorConfiguration: [{ - name: String, - size: Number - }], - dataStorageConfiguration: { - systemRegionInitialSize: Number, - systemRegionMaxSize: Number, - pageSize: Number, - concurrencyLevel: Number, - defaultDataRegionConfiguration: { - name: String, - initialSize: Number, - maxSize: Number, - swapPath: String, - pageEvictionMode: {type: String, enum: ['DISABLED', 'RANDOM_LRU', 'RANDOM_2_LRU']}, - evictionThreshold: Number, - emptyPagesPoolSize: Number, - metricsEnabled: Boolean, - metricsSubIntervalCount: Number, - metricsRateTimeInterval: Number, - persistenceEnabled: Boolean, - checkpointPageBufferSize: Number - }, - dataRegionConfigurations: [{ - name: String, - initialSize: Number, - maxSize: Number, - swapPath: String, - pageEvictionMode: {type: String, enum: ['DISABLED', 'RANDOM_LRU', 'RANDOM_2_LRU']}, - evictionThreshold: Number, - emptyPagesPoolSize: Number, - metricsEnabled: Boolean, - metricsSubIntervalCount: Number, - metricsRateTimeInterval: Number, - persistenceEnabled: Boolean, - checkpointPageBufferSize: Number - }], - storagePath: String, - metricsEnabled: Boolean, - alwaysWriteFullPages: Boolean, - checkpointFrequency: Number, - checkpointThreads: Number, - checkpointWriteOrder: {type: String, enum: ['RANDOM', 'SEQUENTIAL']}, - walPath: String, - walArchivePath: String, - walMode: {type: String, enum: ['DEFAULT', 'LOG_ONLY', 'BACKGROUND', 'NONE']}, - walSegments: Number, - walSegmentSize: Number, - walHistorySize: Number, - walFlushFrequency: Number, - walFsyncDelayNanos: Number, - walRecordIteratorBufferSize: Number, - lockWaitTime: Number, - walBufferSize: Number, - walThreadLocalBufferSize: Number, - metricsSubIntervalCount: Number, - metricsRateTimeInterval: Number, - fileIOFactory: {type: String, enum: ['RANDOM', 'ASYNC']}, - walAutoArchiveAfterInactivity: Number, - writeThrottlingEnabled: Boolean, - walCompactionEnabled: Boolean - }, - memoryConfiguration: { - systemCacheInitialSize: Number, - systemCacheMaxSize: Number, - pageSize: Number, - concurrencyLevel: Number, - defaultMemoryPolicyName: String, - defaultMemoryPolicySize: Number, - memoryPolicies: [{ - name: String, - initialSize: Number, - maxSize: Number, - swapFilePath: String, - pageEvictionMode: {type: String, enum: ['DISABLED', 'RANDOM_LRU', 'RANDOM_2_LRU']}, - evictionThreshold: Number, - emptyPagesPoolSize: Number, - metricsEnabled: Boolean, - subIntervals: Number, - rateTimeInterval: Number - }] - }, - longQueryWarningTimeout: Number, - sqlConnectorConfiguration: { - enabled: Boolean, - host: String, - port: Number, - portRange: Number, - socketSendBufferSize: Number, - socketReceiveBufferSize: Number, - tcpNoDelay: {type: Boolean, default: true}, - maxOpenCursorsPerConnection: Number, - threadPoolSize: Number - }, - persistenceStoreConfiguration: { - enabled: Boolean, - persistentStorePath: String, - metricsEnabled: Boolean, - alwaysWriteFullPages: Boolean, - checkpointingFrequency: Number, - checkpointingPageBufferSize: Number, - checkpointingThreads: Number, - walStorePath: String, - walArchivePath: String, - walMode: {type: String, enum: ['DEFAULT', 'LOG_ONLY', 'BACKGROUND', 'NONE']}, - walSegments: Number, - walSegmentSize: Number, - walHistorySize: Number, - walFlushFrequency: Number, - walFsyncDelayNanos: Number, - walRecordIteratorBufferSize: Number, - lockWaitTime: Number, - rateTimeInterval: Number, - tlbSize: Number, - subIntervals: Number - } - }); - - ClusterSchema.index({name: 1, space: 1}, {unique: true}); - - // Define Cluster model. - result.Cluster = mongoose.model('Cluster', ClusterSchema); - - // Define Notebook schema. - const NotebookSchema = new Schema({ - space: {type: ObjectId, ref: 'Space', index: true, required: true}, - name: String, - expandedParagraphs: [Number], - paragraphs: [{ - name: String, - query: String, - editor: Boolean, - result: {type: String, enum: ['none', 'table', 'bar', 'pie', 'line', 'area']}, - pageSize: Number, - timeLineSpan: String, - maxPages: Number, - hideSystemColumns: Boolean, - cacheName: String, - useAsDefaultSchema: Boolean, - chartsOptions: {barChart: {stacked: Boolean}, areaChart: {style: String}}, - rate: { - value: Number, - unit: Number - }, - qryType: String, - nonCollocatedJoins: {type: Boolean, default: false}, - enforceJoinOrder: {type: Boolean, default: false}, - lazy: {type: Boolean, default: false} - }] - }); - - NotebookSchema.index({name: 1, space: 1}, {unique: true}); - - // Define Notebook model. - result.Notebook = mongoose.model('Notebook', NotebookSchema); - - // Define Activities schema. - const ActivitiesSchema = new Schema({ - owner: {type: ObjectId, ref: 'Account'}, - date: Date, - group: String, - action: String, - amount: { type: Number, default: 0 } - }); - - ActivitiesSchema.index({ owner: 1, group: 1, action: 1, date: 1}, { unique: true }); - - // Define Activities model. - result.Activities = mongoose.model('Activities', ActivitiesSchema); - - // Define Notifications schema. - const NotificationsSchema = new Schema({ - owner: {type: ObjectId, ref: 'Account'}, - date: Date, - message: String, - isShown: Boolean - }); - - // Define Notifications model. - result.Notifications = mongoose.model('Notifications', NotificationsSchema); - result.handleError = function(res, err) { // TODO IGNITE-843 Send error to admin res.status(err.code || 500).send(err.message); @@ -1158,7 +53,7 @@ const defineSchema = (mongoose) => { return result; }; -module.exports.factory = function(settings, mongoose) { +module.exports.factory = function(settings, mongoose, schemas) { // Use native promises mongoose.Promise = global.Promise; @@ -1166,9 +61,9 @@ module.exports.factory = function(settings, mongoose) { // Connect to mongoDB database. return mongoose.connect(settings.mongoUrl, {server: {poolSize: 4}}) - .then(() => defineSchema(mongoose)) - .catch(() => { - console.log('Failed to connect to local MongoDB, will try to download and start embedded MongoDB'); + .then(() => defineSchema(mongoose, schemas)) + .catch((err) => { + console.log('Failed to connect to local MongoDB, will try to download and start embedded MongoDB', err); const {MongodHelper} = require('mongodb-prebuilt'); const {MongoDBDownload} = require('mongodb-download'); @@ -1216,7 +111,7 @@ module.exports.factory = function(settings, mongoose) { return Promise.reject(err); }); }) - .then(() => defineSchema(mongoose)) + .then(() => defineSchema(mongoose, schemas)) .then((mongo) => { if (settings.packaged) { return mongo.Account.count() diff --git a/modules/web-console/backend/app/schemas.js b/modules/web-console/backend/app/schemas.js new file mode 100644 index 0000000000000..0ed85b27f281b --- /dev/null +++ b/modules/web-console/backend/app/schemas.js @@ -0,0 +1,1147 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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'; + +const passportMongo = require('passport-local-mongoose'); + +// Fire me up! + +/** + * Module mongo schema. + */ +module.exports = { + implements: 'schemas', + inject: ['mongoose'] +}; + +module.exports.factory = function(mongoose) { + const Schema = mongoose.Schema; + const ObjectId = mongoose.Schema.Types.ObjectId; + + // Define Account schema. + const Account = new Schema({ + firstName: String, + lastName: String, + email: String, + phone: String, + company: String, + country: String, + registered: Date, + lastLogin: Date, + lastActivity: Date, + admin: Boolean, + token: String, + resetPasswordToken: String + }); + + // Install passport plugin. + Account.plugin(passportMongo, { + usernameField: 'email', limitAttempts: true, lastLoginField: 'lastLogin', + usernameLowerCase: true + }); + + const transform = (doc, ret) => { + return { + _id: ret._id, + email: ret.email, + phone: ret.phone, + firstName: ret.firstName, + lastName: ret.lastName, + company: ret.company, + country: ret.country, + admin: ret.admin, + token: ret.token, + registered: ret.registered, + lastLogin: ret.lastLogin, + lastActivity: ret.lastActivity + }; + }; + + // Configure transformation to JSON. + Account.set('toJSON', {transform}); + + // Configure transformation to JSON. + Account.set('toObject', {transform}); + + // Define Space schema. + const Space = new Schema({ + name: String, + owner: {type: ObjectId, ref: 'Account'}, + demo: {type: Boolean, default: false}, + usedBy: [{ + permission: {type: String, enum: ['VIEW', 'FULL']}, + account: {type: ObjectId, ref: 'Account'} + }] + }); + + // Define Domain model schema. + const DomainModel = new Schema({ + space: {type: ObjectId, ref: 'Space', index: true, required: true}, + caches: [{type: ObjectId, ref: 'Cache'}], + queryMetadata: {type: String, enum: ['Annotations', 'Configuration']}, + kind: {type: String, enum: ['query', 'store', 'both']}, + tableName: String, + keyFieldName: String, + valueFieldName: String, + databaseSchema: String, + databaseTable: String, + keyType: String, + valueType: {type: String}, + keyFields: [{ + databaseFieldName: String, + databaseFieldType: String, + javaFieldName: String, + javaFieldType: String + }], + valueFields: [{ + databaseFieldName: String, + databaseFieldType: String, + javaFieldName: String, + javaFieldType: String + }], + queryKeyFields: [String], + fields: [{name: String, className: String}], + aliases: [{field: String, alias: String}], + indexes: [{ + name: String, + indexType: {type: String, enum: ['SORTED', 'FULLTEXT', 'GEOSPATIAL']}, + fields: [{name: String, direction: Boolean}] + }], + generatePojo: Boolean + }); + + DomainModel.index({valueType: 1, space: 1}, {unique: true}); + + // Define Cache schema. + const Cache = new Schema({ + space: {type: ObjectId, ref: 'Space', index: true, required: true}, + name: {type: String}, + groupName: {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']}, + partitionLossPolicy: { + type: String, + enum: ['READ_ONLY_SAFE', 'READ_ONLY_ALL', 'READ_WRITE_SAFE', 'READ_WRITE_ALL', 'IGNORE'] + }, + + affinity: { + kind: {type: String, enum: ['Default', 'Rendezvous', 'Fair', 'Custom']}, + Rendezvous: { + affinityBackupFilter: String, + partitions: Number, + excludeNeighbors: Boolean + }, + Fair: { + affinityBackupFilter: String, + partitions: Number, + excludeNeighbors: Boolean + }, + Custom: { + className: String + } + }, + + affinityMapper: String, + + nodeFilter: { + kind: {type: String, enum: ['Default', 'Exclude', 'IGFS', 'OnNodes', 'Custom']}, + Exclude: { + nodeId: String + }, + IGFS: { + igfs: {type: ObjectId, ref: 'Igfs'} + }, + Custom: { + className: String + } + }, + + backups: Number, + memoryMode: {type: String, enum: ['ONHEAP_TIERED', 'OFFHEAP_TIERED', 'OFFHEAP_VALUES']}, + offHeapMaxMemory: Number, + startSize: Number, + swapEnabled: Boolean, + + onheapCacheEnabled: Boolean, + + evictionPolicy: { + 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 + } + }, + + rebalanceMode: {type: String, enum: ['SYNC', 'ASYNC', 'NONE']}, + rebalanceBatchSize: Number, + rebalanceBatchesPrefetchCount: Number, + rebalanceOrder: Number, + rebalanceDelay: Number, + rebalanceTimeout: Number, + rebalanceThrottle: Number, + + cacheStoreFactory: { + kind: { + type: String, + enum: ['CacheJdbcPojoStoreFactory', 'CacheJdbcBlobStoreFactory', 'CacheHibernateBlobStoreFactory'] + }, + CacheJdbcPojoStoreFactory: { + dataSourceBean: String, + dialect: { + 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: { + connectVia: {type: String, enum: ['URL', 'DataSource']}, + connectionUrl: String, + user: String, + dataSourceBean: String, + dialect: { + type: String, + enum: ['Generic', 'Oracle', 'DB2', 'SQLServer', 'MySQL', 'PostgreSQL', 'H2'] + }, + initSchema: Boolean, + createTableQuery: String, + loadQuery: String, + insertQuery: String, + updateQuery: String, + deleteQuery: String + }, + CacheHibernateBlobStoreFactory: { + hibernateProperties: [{name: String, value: String}] + } + }, + storeKeepBinary: Boolean, + loadPreviousValue: Boolean, + readThrough: Boolean, + writeThrough: Boolean, + + writeBehindEnabled: Boolean, + writeBehindBatchSize: Number, + writeBehindFlushSize: Number, + writeBehindFlushFrequency: Number, + writeBehindFlushThreadCount: Number, + writeBehindCoalescing: {type: Boolean, default: true}, + + invalidate: Boolean, + defaultLockTimeout: Number, + atomicWriteOrderMode: {type: String, enum: ['CLOCK', 'PRIMARY']}, + writeSynchronizationMode: {type: String, enum: ['FULL_SYNC', 'FULL_ASYNC', 'PRIMARY_SYNC']}, + + sqlEscapeAll: Boolean, + sqlSchema: String, + sqlOnheapRowCacheSize: Number, + longQueryWarningTimeout: Number, + sqlFunctionClasses: [String], + snapshotableIndex: Boolean, + queryDetailMetricsSize: Number, + queryParallelism: Number, + statisticsEnabled: Boolean, + managementEnabled: Boolean, + readFromBackup: Boolean, + copyOnRead: Boolean, + maxConcurrentAsyncOperations: Number, + nearConfiguration: { + 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 + } + } + }, + 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 + } + } + }, + evictionFilter: String, + memoryPolicyName: String, + dataRegionName: String, + sqlIndexMaxInlineSize: Number, + topologyValidator: String + }); + + Cache.index({name: 1, space: 1}, {unique: true}); + + const Igfs = new Schema({ + space: {type: ObjectId, ref: 'Space', index: true, required: true}, + name: {type: String}, + clusters: [{type: ObjectId, ref: 'Cluster'}], + affinnityGroupSize: Number, + blockSize: Number, + streamBufferSize: Number, + dataCacheName: String, + metaCacheName: String, + defaultMode: {type: String, enum: ['PRIMARY', 'PROXY', 'DUAL_SYNC', 'DUAL_ASYNC']}, + dualModeMaxPendingPutsSize: Number, + dualModePutExecutorService: String, + dualModePutExecutorServiceShutdown: Boolean, + fragmentizerConcurrentFiles: Number, + fragmentizerEnabled: Boolean, + fragmentizerThrottlingBlockLength: Number, + fragmentizerThrottlingDelay: Number, + ipcEndpointConfiguration: { + type: {type: String, enum: ['SHMEM', 'TCP']}, + host: String, + port: Number, + memorySize: Number, + tokenDirectoryPath: String, + threadCount: Number + }, + ipcEndpointEnabled: Boolean, + maxSpaceSize: Number, + maximumTaskRangeLength: Number, + managementPort: Number, + pathModes: [{path: String, mode: {type: String, enum: ['PRIMARY', 'PROXY', 'DUAL_SYNC', 'DUAL_ASYNC']}}], + perNodeBatchSize: Number, + perNodeParallelBatchCount: Number, + prefetchBlocks: Number, + sequentialReadsBeforePrefetch: Number, + trashPurgeTimeout: Number, + secondaryFileSystemEnabled: Boolean, + secondaryFileSystem: { + uri: String, + cfgPath: String, + userName: String + }, + colocateMetadata: Boolean, + relaxedConsistency: Boolean, + updateFileLengthOnFlush: Boolean + }); + + Igfs.index({name: 1, space: 1}, {unique: true}); + + + // Define Cluster schema. + const Cluster = new Schema({ + space: {type: ObjectId, ref: 'Space', index: true, required: true}, + name: {type: String}, + activeOnStart: {type: Boolean, default: true}, + localHost: String, + discovery: { + localAddress: String, + localPort: Number, + localPortRange: Number, + addressResolver: String, + socketTimeout: Number, + ackTimeout: Number, + maxAckTimeout: Number, + networkTimeout: Number, + joinTimeout: Number, + threadPriority: Number, + heartbeatFrequency: Number, + maxMissedHeartbeats: Number, + maxMissedClientHeartbeats: Number, + topHistorySize: Number, + listener: String, + dataExchange: String, + metricsProvider: String, + reconnectCount: Number, + statisticsPrintFrequency: Number, + ipFinderCleanFrequency: Number, + authenticator: String, + forceServerMode: Boolean, + clientReconnectDisabled: Boolean, + kind: { + type: String, + enum: ['Vm', 'Multicast', 'S3', 'Cloud', 'GoogleStorage', 'Jdbc', 'SharedFs', 'ZooKeeper', 'Kubernetes'] + }, + Vm: { + addresses: [String] + }, + Multicast: { + multicastGroup: String, + multicastPort: Number, + responseWaitTime: Number, + addressRequestAttempts: Number, + localAddress: String, + addresses: [String] + }, + S3: { + 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, + credentialPath: String, + identity: String, + provider: String, + regions: [String], + zones: [String] + }, + GoogleStorage: { + projectName: String, + bucketName: String, + serviceAccountP12FilePath: String, + serviceAccountId: String, + addrReqAttempts: String + }, + Jdbc: { + initSchema: Boolean, + dataSourceBean: String, + dialect: { + type: String, + enum: ['Generic', 'Oracle', 'DB2', 'SQLServer', 'MySQL', 'PostgreSQL', 'H2'] + } + }, + SharedFs: { + path: String + }, + ZooKeeper: { + curator: String, + zkConnectionString: String, + retryPolicy: { + kind: { + type: String, enum: ['ExponentialBackoff', 'BoundedExponentialBackoff', 'UntilElapsed', + 'NTimes', 'OneTime', 'Forever', 'Custom'] + }, + ExponentialBackoff: { + baseSleepTimeMs: Number, + maxRetries: Number, + maxSleepMs: Number + }, + BoundedExponentialBackoff: { + baseSleepTimeMs: Number, + maxSleepTimeMs: Number, + maxRetries: Number + }, + UntilElapsed: { + maxElapsedTimeMs: Number, + sleepMsBetweenRetries: Number + }, + NTimes: { + n: Number, + sleepMsBetweenRetries: Number + }, + OneTime: { + sleepMsBetweenRetry: Number + }, + Forever: { + retryIntervalMs: Number + }, + Custom: { + className: String + } + }, + basePath: String, + serviceName: String, + allowDuplicateRegistrations: Boolean + }, + Kubernetes: { + serviceName: String, + namespace: String, + masterUrl: String, + accountToken: String + } + }, + atomicConfiguration: { + backups: Number, + cacheMode: {type: String, enum: ['LOCAL', 'REPLICATED', 'PARTITIONED']}, + atomicSequenceReserveSize: Number, + affinity: { + kind: {type: String, enum: ['Default', 'Rendezvous', 'Custom']}, + Rendezvous: { + affinityBackupFilter: String, + partitions: Number, + excludeNeighbors: Boolean + }, + Custom: { + className: String + } + } + }, + binaryConfiguration: { + idMapper: String, + nameMapper: String, + serializer: String, + typeConfigurations: [{ + typeName: String, + idMapper: String, + nameMapper: String, + serializer: String, + enum: Boolean + }], + compactFooter: Boolean + }, + caches: [{type: ObjectId, ref: 'Cache'}], + clockSyncSamples: Number, + clockSyncFrequency: Number, + deploymentMode: {type: String, enum: ['PRIVATE', 'ISOLATED', 'SHARED', 'CONTINUOUS']}, + discoveryStartupDelay: Number, + 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']}, + OptimizedMarshaller: { + poolSize: Number, + requireSerializable: Boolean + } + }, + marshalLocalJobs: Boolean, + marshallerCacheKeepAliveTime: Number, + marshallerCacheThreadPoolSize: Number, + metricsExpireTime: Number, + metricsHistorySize: Number, + metricsLogFrequency: Number, + metricsUpdateFrequency: Number, + networkTimeout: Number, + networkSendRetryDelay: Number, + networkSendRetryCount: Number, + communication: { + listener: String, + localAddress: String, + localPort: Number, + localPortRange: Number, + sharedMemoryPort: Number, + directBuffer: Boolean, + directSendBuffer: Boolean, + idleConnectionTimeout: Number, + connectTimeout: Number, + maxConnectTimeout: Number, + reconnectCount: Number, + socketSendBuffer: Number, + socketReceiveBuffer: Number, + messageQueueLimit: Number, + slowClientQueueLimit: Number, + tcpNoDelay: Boolean, + ackSendThreshold: Number, + unacknowledgedMessagesBufferSize: Number, + socketWriteTimeout: Number, + selectorsCount: Number, + addressResolver: String + }, + connector: { + enabled: Boolean, + jettyPath: String, + host: String, + port: Number, + portRange: Number, + idleTimeout: Number, + idleQueryCursorTimeout: Number, + idleQueryCursorCheckFrequency: Number, + receiveBufferSize: Number, + sendBufferSize: Number, + sendQueueLimit: Number, + directBuffer: Boolean, + noDelay: Boolean, + selectorCount: Number, + threadPoolSize: Number, + messageInterceptor: String, + secretKey: String, + sslEnabled: Boolean, + sslClientAuth: Boolean, + sslFactory: String + }, + peerClassLoadingEnabled: Boolean, + peerClassLoadingLocalClassPathExclude: [String], + peerClassLoadingMissedResourcesCacheSize: Number, + peerClassLoadingThreadPoolSize: Number, + publicThreadPoolSize: Number, + swapSpaceSpi: { + kind: {type: String, enum: ['FileSwapSpaceSpi']}, + FileSwapSpaceSpi: { + baseDirectory: String, + readStripesNumber: Number, + maximumSparsity: Number, + maxWriteQueueSize: Number, + writeBufferSize: Number + } + }, + systemThreadPoolSize: Number, + timeServerPortBase: Number, + timeServerPortRange: Number, + transactionConfiguration: { + defaultTxConcurrency: {type: String, enum: ['OPTIMISTIC', 'PESSIMISTIC']}, + defaultTxIsolation: {type: String, enum: ['READ_COMMITTED', 'REPEATABLE_READ', 'SERIALIZABLE']}, + defaultTxTimeout: Number, + pessimisticTxLogLinger: Number, + pessimisticTxLogSize: Number, + txSerializableEnabled: Boolean, + txManagerFactory: String + }, + sslEnabled: Boolean, + sslContextFactory: { + keyAlgorithm: String, + keyStoreFilePath: String, + keyStoreType: String, + protocol: String, + trustStoreFilePath: String, + trustStoreType: String, + trustManagers: [String] + }, + rebalanceThreadPoolSize: Number, + odbc: { + odbcEnabled: Boolean, + endpointAddress: String, + socketSendBufferSize: Number, + socketReceiveBufferSize: Number, + maxOpenCursors: Number, + threadPoolSize: Number + }, + attributes: [{name: String, value: String}], + collision: { + kind: {type: String, enum: ['Noop', 'PriorityQueue', 'FifoQueue', 'JobStealing', 'Custom']}, + PriorityQueue: { + parallelJobsNumber: Number, + waitingJobsNumber: Number, + priorityAttributeKey: String, + jobPriorityAttributeKey: String, + defaultPriority: Number, + starvationIncrement: Number, + starvationPreventionEnabled: Boolean + }, + FifoQueue: { + parallelJobsNumber: Number, + waitingJobsNumber: Number + }, + JobStealing: { + activeJobsThreshold: Number, + waitJobsThreshold: Number, + messageExpireTime: Number, + maximumStealingAttempts: Number, + stealingEnabled: Boolean, + stealingAttributes: [{name: String, value: String}], + externalCollisionListener: String + }, + Custom: { + class: String + } + }, + failoverSpi: [{ + kind: {type: String, enum: ['JobStealing', 'Never', 'Always', 'Custom']}, + JobStealing: { + maximumFailoverAttempts: Number + }, + Always: { + maximumFailoverAttempts: Number + }, + Custom: { + class: String + } + }], + logger: { + kind: {type: 'String', enum: ['Log4j2', 'Null', 'Java', 'JCL', 'SLF4J', 'Log4j', 'Custom']}, + Log4j2: { + level: {type: String, enum: ['OFF', 'FATAL', 'ERROR', 'WARN', 'INFO', 'DEBUG', 'TRACE', 'ALL']}, + path: String + }, + Log4j: { + mode: {type: String, enum: ['Default', 'Path']}, + level: {type: String, enum: ['OFF', 'FATAL', 'ERROR', 'WARN', 'INFO', 'DEBUG', 'TRACE', 'ALL']}, + path: String + }, + Custom: { + class: String + } + }, + 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 + } + }], + clientConnectorConfiguration: { + enabled: Boolean, + host: String, + port: Number, + portRange: Number, + socketSendBufferSize: Number, + socketReceiveBufferSize: Number, + tcpNoDelay: {type: Boolean, default: true}, + maxOpenCursorsPerConnection: Number, + threadPoolSize: Number + }, + 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 + } + }], + 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 + } + }, + warmupClosure: String, + hadoopConfiguration: { + mapReducePlanner: { + kind: {type: String, enum: ['Weighted', 'Custom']}, + Weighted: { + localMapperWeight: Number, + remoteMapperWeight: Number, + localReducerWeight: Number, + remoteReducerWeight: Number, + preferLocalReducerThresholdWeight: Number + }, + Custom: { + className: String + } + }, + finishedJobInfoTtl: Number, + maxParallelTasks: Number, + maxTaskQueueSize: Number, + nativeLibraryNames: [String] + }, + serviceConfigurations: [{ + name: String, + service: String, + maxPerNodeCount: Number, + totalCount: Number, + nodeFilter: { + kind: {type: String, enum: ['Default', 'Exclude', 'IGFS', 'OnNodes', 'Custom']}, + Exclude: { + nodeId: String + }, + IGFS: { + igfs: {type: ObjectId, ref: 'Igfs'} + }, + Custom: { + className: String + } + }, + cache: {type: ObjectId, ref: 'Cache'}, + affinityKey: String + }], + cacheSanityCheckEnabled: {type: Boolean, default: true}, + classLoader: String, + consistentId: String, + failureDetectionTimeout: Number, + clientFailureDetectionTimeout: Number, + workDirectory: String, + lateAffinityAssignment: Boolean, + utilityCacheKeepAliveTime: Number, + asyncCallbackPoolSize: Number, + dataStreamerThreadPoolSize: Number, + queryThreadPoolSize: Number, + stripedPoolSize: Number, + serviceThreadPoolSize: Number, + utilityCacheThreadPoolSize: Number, + executorConfiguration: [{ + name: String, + size: Number + }], + dataStorageConfiguration: { + systemRegionInitialSize: Number, + systemRegionMaxSize: Number, + pageSize: Number, + concurrencyLevel: Number, + defaultDataRegionConfiguration: { + name: String, + initialSize: Number, + maxSize: Number, + swapPath: String, + pageEvictionMode: {type: String, enum: ['DISABLED', 'RANDOM_LRU', 'RANDOM_2_LRU']}, + evictionThreshold: Number, + emptyPagesPoolSize: Number, + metricsEnabled: Boolean, + metricsSubIntervalCount: Number, + metricsRateTimeInterval: Number, + persistenceEnabled: Boolean, + checkpointPageBufferSize: Number + }, + dataRegionConfigurations: [{ + name: String, + initialSize: Number, + maxSize: Number, + swapPath: String, + pageEvictionMode: {type: String, enum: ['DISABLED', 'RANDOM_LRU', 'RANDOM_2_LRU']}, + evictionThreshold: Number, + emptyPagesPoolSize: Number, + metricsEnabled: Boolean, + metricsSubIntervalCount: Number, + metricsRateTimeInterval: Number, + persistenceEnabled: Boolean, + checkpointPageBufferSize: Number + }], + storagePath: String, + metricsEnabled: Boolean, + alwaysWriteFullPages: Boolean, + checkpointFrequency: Number, + checkpointThreads: Number, + checkpointWriteOrder: {type: String, enum: ['RANDOM', 'SEQUENTIAL']}, + walPath: String, + walArchivePath: String, + walMode: {type: String, enum: ['DEFAULT', 'LOG_ONLY', 'BACKGROUND', 'NONE']}, + walSegments: Number, + walSegmentSize: Number, + walHistorySize: Number, + walFlushFrequency: Number, + walFsyncDelayNanos: Number, + walRecordIteratorBufferSize: Number, + lockWaitTime: Number, + walBufferSize: Number, + walThreadLocalBufferSize: Number, + metricsSubIntervalCount: Number, + metricsRateTimeInterval: Number, + fileIOFactory: {type: String, enum: ['RANDOM', 'ASYNC']}, + walAutoArchiveAfterInactivity: Number, + writeThrottlingEnabled: Boolean, + walCompactionEnabled: Boolean + }, + memoryConfiguration: { + systemCacheInitialSize: Number, + systemCacheMaxSize: Number, + pageSize: Number, + concurrencyLevel: Number, + defaultMemoryPolicyName: String, + defaultMemoryPolicySize: Number, + memoryPolicies: [{ + name: String, + initialSize: Number, + maxSize: Number, + swapFilePath: String, + pageEvictionMode: {type: String, enum: ['DISABLED', 'RANDOM_LRU', 'RANDOM_2_LRU']}, + evictionThreshold: Number, + emptyPagesPoolSize: Number, + metricsEnabled: Boolean, + subIntervals: Number, + rateTimeInterval: Number + }] + }, + longQueryWarningTimeout: Number, + sqlConnectorConfiguration: { + enabled: Boolean, + host: String, + port: Number, + portRange: Number, + socketSendBufferSize: Number, + socketReceiveBufferSize: Number, + tcpNoDelay: {type: Boolean, default: true}, + maxOpenCursorsPerConnection: Number, + threadPoolSize: Number + }, + persistenceStoreConfiguration: { + enabled: Boolean, + persistentStorePath: String, + metricsEnabled: Boolean, + alwaysWriteFullPages: Boolean, + checkpointingFrequency: Number, + checkpointingPageBufferSize: Number, + checkpointingThreads: Number, + walStorePath: String, + walArchivePath: String, + walMode: {type: String, enum: ['DEFAULT', 'LOG_ONLY', 'BACKGROUND', 'NONE']}, + walSegments: Number, + walSegmentSize: Number, + walHistorySize: Number, + walFlushFrequency: Number, + walFsyncDelayNanos: Number, + walRecordIteratorBufferSize: Number, + lockWaitTime: Number, + rateTimeInterval: Number, + tlbSize: Number, + subIntervals: Number + } + }); + + Cluster.index({name: 1, space: 1}, {unique: true}); + + // Define Notebook schema. + const Notebook = new Schema({ + space: {type: ObjectId, ref: 'Space', index: true, required: true}, + name: String, + expandedParagraphs: [Number], + paragraphs: [{ + name: String, + query: String, + editor: Boolean, + result: {type: String, enum: ['none', 'table', 'bar', 'pie', 'line', 'area']}, + pageSize: Number, + timeLineSpan: String, + maxPages: Number, + hideSystemColumns: Boolean, + cacheName: String, + useAsDefaultSchema: Boolean, + chartsOptions: {barChart: {stacked: Boolean}, areaChart: {style: String}}, + rate: { + value: Number, + unit: Number + }, + qryType: String, + nonCollocatedJoins: {type: Boolean, default: false}, + enforceJoinOrder: {type: Boolean, default: false}, + lazy: {type: Boolean, default: false} + }] + }); + + Notebook.index({name: 1, space: 1}, {unique: true}); + + // Define Activities schema. + const Activities = new Schema({ + owner: {type: ObjectId, ref: 'Account'}, + date: Date, + group: String, + action: String, + amount: {type: Number, default: 0} + }); + + Activities.index({owner: 1, group: 1, action: 1, date: 1}, {unique: true}); + + // Define Notifications schema. + const Notifications = new Schema({ + owner: {type: ObjectId, ref: 'Account'}, + date: Date, + message: String, + isShown: Boolean + }); + + return { + Space, + Account, + DomainModel, + Cache, + Igfs, + Cluster, + Notebook, + Activities, + Notifications + }; +}; diff --git a/modules/web-console/backend/services/users.js b/modules/web-console/backend/services/users.js index 93e752f4df5ca..43bceda3cd6d5 100644 --- a/modules/web-console/backend/services/users.js +++ b/modules/web-console/backend/services/users.js @@ -135,6 +135,7 @@ module.exports.factory = (errors, settings, mongo, spacesService, mailsService, /** * Get list of user accounts and summary information. + * * @returns {mongo.Account[]} - returns all accounts with counters object */ static list(params) { From 89e15830dedcb46f24d9cc9b922ba3b013331a18 Mon Sep 17 00:00:00 2001 From: Vasiliy Sisko Date: Mon, 12 Feb 2018 17:22:10 +0700 Subject: [PATCH 043/314] Web Agent: Fixed wrong config of IGNITE_ATOMIC_CACHE_DELETE_HISTORY_SIZE in demo startup. (cherry picked from commit 1a6e544) --- .../java/org/apache/ignite/console/demo/AgentClusterDemo.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 7157df7e8c65d..6bfc5bd16a257 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 @@ -184,7 +184,7 @@ public static CountDownLatch tryStart() { System.setProperty(IGNITE_QUIET, "false"); System.setProperty(IGNITE_UPDATE_NOTIFIER, "false"); - System.setProperty(IGNITE_ATOMIC_CACHE_DELETE_HISTORY_SIZE, "1"); + System.setProperty(IGNITE_ATOMIC_CACHE_DELETE_HISTORY_SIZE, "20"); System.setProperty(IGNITE_PERFORMANCE_SUGGESTIONS_DISABLED, "true"); final AtomicInteger basePort = new AtomicInteger(60700); From 18966673570425192e1b89fbb2c63d164b47eaca Mon Sep 17 00:00:00 2001 From: Vasiliy Sisko Date: Mon, 12 Feb 2018 20:24:30 +0700 Subject: [PATCH 044/314] IGNITE-7578 Actualized client connector configuration. (cherry picked from commit 819d746) --- .../VisorClientConnectorConfiguration.java | 117 +++++++++++++++++- .../config/VisorConfigurationCommand.scala | 8 ++ modules/web-console/backend/app/schemas.js | 10 +- .../generator/ConfigurationGenerator.js | 14 +++ .../generator/defaults/Cluster.service.js | 15 +++ .../clusters/client-connector.pug | 33 ++++- .../frontend/app/services/Version.service.js | 4 + .../controllers/clusters-controller.js | 9 +- 8 files changed, 202 insertions(+), 8 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorClientConnectorConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorClientConnectorConfiguration.java index 397b72a3c0002..0a1459ee0d01e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorClientConnectorConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorClientConnectorConfiguration.java @@ -26,6 +26,8 @@ import org.apache.ignite.internal.visor.VisorDataTransferObject; import org.jetbrains.annotations.Nullable; +import static org.apache.ignite.internal.visor.util.VisorTaskUtils.compactClass; + /** * Data transfer object for client connector configuration. */ @@ -55,7 +57,31 @@ public class VisorClientConnectorConfiguration extends VisorDataTransferObject { private boolean tcpNoDelay; /** Thread pool size. */ - private int threadPoolSize ; + private int threadPoolSize; + + /** Idle timeout. */ + private long idleTimeout; + + /** JDBC connections enabled flag. */ + private boolean jdbcEnabled; + + /** ODBC connections enabled flag. */ + private boolean odbcEnabled; + + /** JDBC connections enabled flag. */ + private boolean thinCliEnabled; + + /** SSL enable flag, default is disabled. */ + private boolean sslEnabled; + + /** If to use SSL context factory from Ignite configuration. */ + private boolean useIgniteSslCtxFactory; + + /** SSL need client auth flag. */ + private boolean sslClientAuth; + + /** SSL connection factory class name. */ + private String sslCtxFactory; /** * Default constructor. @@ -78,6 +104,14 @@ public VisorClientConnectorConfiguration(ClientConnectorConfiguration cfg) { sockRcvBufSize = cfg.getSocketReceiveBufferSize(); tcpNoDelay = cfg.isTcpNoDelay(); threadPoolSize = cfg.getThreadPoolSize(); + idleTimeout = cfg.getIdleTimeout(); + jdbcEnabled = cfg.isJdbcEnabled(); + odbcEnabled = cfg.isOdbcEnabled(); + thinCliEnabled = cfg.isThinClientEnabled(); + sslEnabled = cfg.isSslEnabled(); + useIgniteSslCtxFactory = cfg.isUseIgniteSslContextFactory(); + sslClientAuth = cfg.isSslClientAuth(); + sslCtxFactory = compactClass(cfg.getSslContextFactory()); } /** @@ -135,6 +169,66 @@ public int getThreadPoolSize() { return threadPoolSize; } + /** + * @return Idle timeout. + */ + public long getIdleTimeout() { + return idleTimeout; + } + + /** + * @return JDBC connections enabled flag. + */ + public boolean isJdbcEnabled() { + return jdbcEnabled; + } + + /** + * @return ODBC connections enabled flag. + */ + public boolean isOdbcEnabled() { + return odbcEnabled; + } + + /** + * @return JDBC connections enabled flag. + */ + public boolean isThinClientEnabled() { + return thinCliEnabled; + } + + /** + * @return SSL enable flag, default is disabled. + */ + public boolean isSslEnabled() { + return sslEnabled; + } + + /** + * @return If to use SSL context factory from Ignite configuration. + */ + public boolean isUseIgniteSslContextFactory() { + return useIgniteSslCtxFactory; + } + + /** + * @return SSL need client auth flag. + */ + public boolean isSslClientAuth() { + return sslClientAuth; + } + + /** + * @return SSL connection factory. + */ + public String getSslContextFactory() { + return sslCtxFactory; + } + + @Override public byte getProtocolVersion() { + return V2; + } + /** {@inheritDoc} */ @Override protected void writeExternalData(ObjectOutput out) throws IOException { U.writeString(out, host); @@ -142,9 +236,17 @@ public int getThreadPoolSize() { out.writeInt(portRange); out.writeInt(maxOpenCursorsPerConn); out.writeInt(sockSndBufSize); - out.writeInt(sockRcvBufSize ); + out.writeInt(sockRcvBufSize); out.writeBoolean(tcpNoDelay); out.writeInt(threadPoolSize); + out.writeLong(idleTimeout); + out.writeBoolean(jdbcEnabled); + out.writeBoolean(odbcEnabled); + out.writeBoolean(thinCliEnabled); + out.writeBoolean(sslEnabled); + out.writeBoolean(useIgniteSslCtxFactory); + out.writeBoolean(sslClientAuth); + U.writeString(out, sslCtxFactory); } /** {@inheritDoc} */ @@ -157,6 +259,17 @@ public int getThreadPoolSize() { sockRcvBufSize = in.readInt(); tcpNoDelay = in.readBoolean(); threadPoolSize = in.readInt(); + + if (protoVer > V1) { + idleTimeout = in.readLong(); + jdbcEnabled = in.readBoolean(); + odbcEnabled = in.readBoolean(); + thinCliEnabled = in.readBoolean(); + sslEnabled = in.readBoolean(); + useIgniteSslCtxFactory = in.readBoolean(); + sslClientAuth = in.readBoolean(); + sslCtxFactory = U.readString(in); + } } /** {@inheritDoc} */ diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/config/VisorConfigurationCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/config/VisorConfigurationCommand.scala index 67d9c142cf68f..1a82381013a1b 100644 --- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/config/VisorConfigurationCommand.scala +++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/config/VisorConfigurationCommand.scala @@ -245,7 +245,15 @@ class VisorConfigurationCommand extends VisorConsoleCommand { cliConnTbl += ("Socket receive buffer size", formatMemory(cliConnCfg.getSocketReceiveBufferSize)) cliConnTbl += ("Max connection cursors", cliConnCfg.getMaxOpenCursorsPerConnection) cliConnTbl += ("Pool size", cliConnCfg.getThreadPoolSize) + cliConnTbl += ("Idle Timeout", cliConnCfg.getIdleTimeout + "ms") cliConnTbl += ("TCP_NODELAY", bool2Str(cliConnCfg.isTcpNoDelay)) + cliConnTbl += ("JDBC Enabled", bool2Str(cliConnCfg.isJdbcEnabled)) + cliConnTbl += ("ODBC Enabled", bool2Str(cliConnCfg.isOdbcEnabled)) + cliConnTbl += ("Thin Client Enabled", bool2Str(cliConnCfg.isThinClientEnabled)) + cliConnTbl += ("SSL Enabled", bool2Str(cliConnCfg.isSslEnabled)) + cliConnTbl += ("Ssl Client Auth", bool2Str(cliConnCfg.isSslClientAuth)) + cliConnTbl += ("Use Ignite SSL Context Factory", bool2Str(cliConnCfg.isUseIgniteSslContextFactory)) + cliConnTbl += ("SSL Context Factory", safe(cliConnCfg.getSslContextFactory)) cliConnTbl.render() } diff --git a/modules/web-console/backend/app/schemas.js b/modules/web-console/backend/app/schemas.js index 0ed85b27f281b..b3d61ac44c8e6 100644 --- a/modules/web-console/backend/app/schemas.js +++ b/modules/web-console/backend/app/schemas.js @@ -859,7 +859,15 @@ module.exports.factory = function(mongoose) { socketReceiveBufferSize: Number, tcpNoDelay: {type: Boolean, default: true}, maxOpenCursorsPerConnection: Number, - threadPoolSize: Number + threadPoolSize: Number, + idleTimeout: Number, + jdbcEnabled: {type: Boolean, default: true}, + odbcEnabled: {type: Boolean, default: true}, + thinClientEnabled: {type: Boolean, default: true}, + sslEnabled: Boolean, + useIgniteSslContextFactory: {type: Boolean, default: true}, + sslClientAuth: Boolean, + sslContextFactory: String }, loadBalancingSpi: [{ kind: {type: String, enum: ['RoundRobin', 'Adaptive', 'WeightedRandom', 'Custom']}, 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 9269fc930f74f..645e3e34b05ee 100644 --- a/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js @@ -788,6 +788,20 @@ export default class IgniteConfigurationGenerator { .intProperty('threadPoolSize') .boolProperty('tcpNoDelay'); + if (available('2.4.0')) { + bean.longProperty('idleTimeout') + .boolProperty('jdbcEnabled') + .boolProperty('odbcEnabled') + .boolProperty('thinClientEnabled'); + } + + if (available('2.5.0')) { + bean.boolProperty('sslEnabled') + .boolProperty('sslClientAuth') + .boolProperty('useIgniteSslContextFactory') + .emptyBeanProperty('sslContextFactory'); + } + cfg.beanProperty('clientConnectorConfiguration', bean); return cfg; diff --git a/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.service.js b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.service.js index f636869d99578..b55ed838eb979 100644 --- a/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.service.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.service.js @@ -395,6 +395,21 @@ const DFLT_CLUSTER = { socketReceiveBufferSize: 0, tcpNoDelay: true, maxOpenCursorsPerConnection: 128 + }, + clientConnectorConfiguration: { + port: 10800, + portRange: 100, + socketSendBufferSize: 0, + socketReceiveBufferSize: 0, + tcpNoDelay: true, + maxOpenCursorsPerConnection: 128, + idleTimeout: 0, + jdbcEnabled: true, + odbcEnabled: true, + thinClientEnabled: true, + sslEnabled: false, + useIgniteSslContextFactory: true, + sslClientAuth: false } }; diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/client-connector.pug b/modules/web-console/frontend/app/modules/states/configuration/clusters/client-connector.pug index dd2fa6dedc4cf..c90cc45500e63 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/client-connector.pug +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/client-connector.pug @@ -18,8 +18,10 @@ include /app/helpers/jade/mixins -var form = 'clientConnector' -var model = 'backupItem' --var connectionModel = model + '.clientConnectorConfiguration' --var connectionEnabled = connectionModel + '.enabled' +-var connectionModel = `${model}.clientConnectorConfiguration` +-var connectionEnabled = `${connectionModel}.enabled` +-var sslEnabled = `${connectionEnabled} && ${connectionModel}.sslEnabled` +-var sslFactoryEnabled = `${sslEnabled} && !${connectionModel}.useIgniteSslContextFactory` .panel.panel-default(ng-show='$ctrl.available("2.3.0")' ng-form=form novalidate) .panel-heading(bs-collapse-toggle ng-click=`ui.loadPanel('${form}')`) @@ -41,11 +43,11 @@ include /app/helpers/jade/mixins +number('Port range:', `${connectionModel}.portRange`, '"ClientConnectorPortRange"', connectionEnabled, '100', '0', 'Port range') .settings-row +number('Socket send buffer size:', `${connectionModel}.socketSendBufferSize`, '"ClientConnectorSocketSendBufferSize"', connectionEnabled, '0', '0', - 'Socket send buffer size.
\ + 'Socket send buffer size
\ When set to 0, operation system default will be used') .settings-row +number('Socket receive buffer size:', `${connectionModel}.socketReceiveBufferSize`, '"ClientConnectorSocketReceiveBufferSize"', connectionEnabled, '0', '0', - 'Socket receive buffer size.
\ + 'Socket receive buffer size
\ When set to 0, operation system default will be used') .settings-row +number('Max connection cursors:', `${connectionModel}.maxOpenCursorsPerConnection`, '"ClientConnectorMaxOpenCursorsPerConnection"', connectionEnabled, '128', '0', @@ -53,7 +55,30 @@ include /app/helpers/jade/mixins .settings-row +number('Pool size:', `${connectionModel}.threadPoolSize`, '"ClientConnectorThreadPoolSize"', connectionEnabled, 'max(8, availableProcessors)', '1', 'Size of thread pool that is in charge of processing SQL requests') + .settings-row(ng-if='$ctrl.available("2.4.0")') + +number('Idle timeout:', `${connectionModel}.idleTimeout`, '"ClientConnectorIdleTimeout"', connectionEnabled, '0', '-1', + 'Idle timeout for client connections
\ + Zero or negative means no timeout') + div(ng-if='$ctrl.available("2.5.0")') + .settings-row + +checkbox-enabled('Enable SSL', `${connectionModel}.sslEnabled`, '"ClientConnectorSslEnabled"', connectionEnabled, 'Enable secure socket layer on client connector') + .settings-row + +checkbox-enabled('Enable SSL client auth', `${connectionModel}.sslClientAuth`, '"ClientConnectorSslClientAuth"', sslEnabled, 'Flag indicating whether or not SSL client authentication is required') + .settings-row + +checkbox-enabled('Use Ignite SSL', `${connectionModel}.useIgniteSslContextFactory`, '"ClientConnectorUseIgniteSslContextFactory"', sslEnabled, 'Use SSL factory Ignite configuration') + .settings-row + +java-class('SSL factory:', `${connectionModel}.sslContextFactory`, '"ClientConnectorSslContextFactory"', sslFactoryEnabled, sslFactoryEnabled, + 'If SSL factory specified then replication will be performed through secure SSL channel created with this factory
\ + If not present isUseIgniteSslContextFactory() flag will be evaluated
\ + If set to true and IgniteConfiguration#getSslContextFactory() exists, then Ignite SSL context factory will be used to establish secure connection') .settings-row +checkbox-enabled('TCP_NODELAY option', `${connectionModel}.tcpNoDelay`, '"ClientConnectorTcpNoDelay"', connectionEnabled, 'Value for TCP_NODELAY socket option') + div(ng-if='$ctrl.available("2.4.0")') + .settings-row + +checkbox-enabled('JDBC Enabled', `${connectionModel}.jdbcEnabled`, '"ClientConnectorJdbcEnabled"', connectionEnabled, 'Access through JDBC is enabled') + .settings-row + +checkbox-enabled('ODBC Enabled', `${connectionModel}.odbcEnabled`, '"ClientConnectorOdbcEnabled"', connectionEnabled, 'Access through ODBC is enabled') + .settings-row + +checkbox-enabled('Thin client enabled', `${connectionModel}.thinClientEnabled`, '"ClientConnectorThinCliEnabled"', connectionEnabled, 'Access through thin client is enabled') .col-sm-6 +preview-xml-java(model, 'clusterClientConnector') diff --git a/modules/web-console/frontend/app/services/Version.service.js b/modules/web-console/frontend/app/services/Version.service.js index 9bd9c7dfc302d..88e403b7a0b79 100644 --- a/modules/web-console/frontend/app/services/Version.service.js +++ b/modules/web-console/frontend/app/services/Version.service.js @@ -76,6 +76,10 @@ export default class IgniteVersion { this.webConsole = '2.4.0'; this.supportedVersions = [ + { + label: 'Ignite 2.5', + ignite: '2.5.0' + }, { label: 'Ignite 2.4', ignite: '2.4.0' diff --git a/modules/web-console/frontend/controllers/clusters-controller.js b/modules/web-console/frontend/controllers/clusters-controller.js index 2485fa69e9397..7d987dde3e00e 100644 --- a/modules/web-console/frontend/controllers/clusters-controller.js +++ b/modules/web-console/frontend/controllers/clusters-controller.js @@ -468,7 +468,14 @@ export default ['$rootScope', '$scope', '$http', '$state', '$timeout', 'IgniteLe failoverSpi: [], logger: {Log4j: { mode: 'Default'}}, caches: linkId && _.find($scope.caches, {value: linkId}) ? [linkId] : [], - igfss: linkId && _.find($scope.igfss, {value: linkId}) ? [linkId] : [] + igfss: linkId && _.find($scope.igfss, {value: linkId}) ? [linkId] : [], + clientConnectorConfiguration: { + tcpNoDelay: true, + jdbcEnabled: true, + odbcEnabled: true, + thinClientEnabled: true, + useIgniteSslContextFactory: true + } }); } From 237063efa35c54bb9e9800ecf63ea223ec20a9ef Mon Sep 17 00:00:00 2001 From: alexdel Date: Mon, 19 Feb 2018 11:25:24 +0700 Subject: [PATCH 045/314] IGNITE-7650 Extracted signin/signup form to separate page, improved landing page. (cherry picked from commit 1925674) --- modules/web-console/e2e/testcafe/Dockerfile | 2 + modules/web-console/e2e/testcafe/envtools.js | 1 + .../web-console/e2e/testcafe/fixtures/auth.js | 73 +++---- .../fixtures/user-profile/credentials.js | 63 ++++++ .../profile.js} | 53 ++--- modules/web-console/e2e/testcafe/roles.js | 23 +- modules/web-console/e2e/testcafe/testcafe.js | 4 +- modules/web-console/e2e/testenv/Dockerfile | 2 + modules/web-console/frontend/app/app.js | 14 +- .../app/components/page-configure/index.js | 3 + .../app/components/page-landing/index.js | 65 ++++++ .../app/components/page-landing/style.scss | 104 +++++++++ .../app/components/page-landing/template.pug | 66 ++++++ .../page-password-changed/controller.js | 18 ++ .../components/page-password-changed/index.js | 33 +++ .../page-password-changed/style.scss | 34 +++ .../page-password-changed/template.pug | 14 ++ .../page-password-reset/controller.js | 53 +++++ .../components/page-password-reset/index.js | 57 +++++ .../components/page-password-reset/style.scss | 31 +++ .../page-password-reset/template.pug | 80 +++++++ .../app/components/page-profile/controller.js | 8 +- .../app/components/page-profile/index.js | 13 +- .../app/components/page-profile/style.scss | 31 +++ .../app/components/page-profile/template.pug | 203 ++++++++++++------ .../page-signin/controller.js} | 17 +- .../app/components/page-signin/index.js | 56 +++++ .../app/components/page-signin/style.scss | 38 ++++ .../app/components/page-signin/template.pug | 161 ++++++++++++++ .../components/web-console-header/style.scss | 1 + .../web-console-header/template.pug | 2 +- .../frontend/app/data/countries.json | 72 ++++--- .../frontend/app/helpers/jade/mixins.pug | 1 + .../app/modules/states/configuration.state.js | 1 + .../app/modules/states/signin.state.js | 53 ----- .../frontend/app/modules/user/Auth.service.js | 6 +- .../app/primitives/form-field/checkbox.pug | 30 +++ .../app/primitives/form-field/dropdown.pug | 51 +++++ .../app/primitives/form-field/email.pug | 37 ++++ .../app/primitives/form-field/error.pug | 29 +++ .../app/primitives/form-field/index.pug | 27 +++ .../app/primitives/form-field/index.scss | 149 ++++++++++++- .../app/primitives/form-field/input.pug | 28 +++ .../app/primitives/form-field/label.pug | 30 +++ .../app/primitives/form-field/number.pug | 46 ++++ .../app/primitives/form-field/password.pug | 37 ++++ .../app/primitives/form-field/phone.pug | 35 +++ .../app/primitives/form-field/text.pug | 35 +++ .../app/primitives/form-field/tooltip.pug | 18 ++ .../frontend/app/primitives/panel/index.scss | 96 +++++++-- .../app/primitives/tooltip/index.scss | 17 ++ .../app/primitives/ui-grid/index.scss | 32 +-- .../DefaultState.js} | 46 ++-- .../frontend/app/services/index.js | 2 + modules/web-console/frontend/app/vendor.js | 5 +- modules/web-console/frontend/package.json | 3 +- .../frontend/public/images/icons/collapse.svg | 3 + .../public/images/icons/exclamation.svg | 3 + .../frontend/public/images/icons/expand.svg | 3 + .../frontend/public/images/icons/index.js | 4 + .../public/images/main-screenshot.png | Bin 0 -> 86830 bytes .../public/images/page-landing-carousel-1.png | Bin 0 -> 84434 bytes .../public/images/page-landing-carousel-2.png | Bin 0 -> 106670 bytes .../public/images/page-landing-carousel-3.png | Bin 0 -> 70876 bytes .../public/stylesheets/_bootstrap-custom.scss | 1 - .../web-console/frontend/views/reset.tpl.pug | 44 ---- .../web-console/frontend/views/signin.tpl.pug | 165 -------------- 67 files changed, 1893 insertions(+), 539 deletions(-) create mode 100644 modules/web-console/e2e/testcafe/fixtures/user-profile/credentials.js rename modules/web-console/e2e/testcafe/fixtures/{user-profile.js => user-profile/profile.js} (61%) create mode 100644 modules/web-console/frontend/app/components/page-landing/index.js create mode 100644 modules/web-console/frontend/app/components/page-landing/style.scss create mode 100644 modules/web-console/frontend/app/components/page-landing/template.pug create mode 100644 modules/web-console/frontend/app/components/page-password-changed/controller.js create mode 100644 modules/web-console/frontend/app/components/page-password-changed/index.js create mode 100644 modules/web-console/frontend/app/components/page-password-changed/style.scss create mode 100644 modules/web-console/frontend/app/components/page-password-changed/template.pug create mode 100644 modules/web-console/frontend/app/components/page-password-reset/controller.js create mode 100644 modules/web-console/frontend/app/components/page-password-reset/index.js create mode 100644 modules/web-console/frontend/app/components/page-password-reset/style.scss create mode 100644 modules/web-console/frontend/app/components/page-password-reset/template.pug create mode 100644 modules/web-console/frontend/app/components/page-profile/style.scss rename modules/web-console/frontend/app/{controllers/auth.controller.js => components/page-signin/controller.js} (71%) create mode 100644 modules/web-console/frontend/app/components/page-signin/index.js create mode 100644 modules/web-console/frontend/app/components/page-signin/style.scss create mode 100644 modules/web-console/frontend/app/components/page-signin/template.pug delete mode 100644 modules/web-console/frontend/app/modules/states/signin.state.js create mode 100644 modules/web-console/frontend/app/primitives/form-field/checkbox.pug create mode 100644 modules/web-console/frontend/app/primitives/form-field/dropdown.pug create mode 100644 modules/web-console/frontend/app/primitives/form-field/email.pug create mode 100644 modules/web-console/frontend/app/primitives/form-field/error.pug create mode 100644 modules/web-console/frontend/app/primitives/form-field/index.pug create mode 100644 modules/web-console/frontend/app/primitives/form-field/input.pug create mode 100644 modules/web-console/frontend/app/primitives/form-field/label.pug create mode 100644 modules/web-console/frontend/app/primitives/form-field/number.pug create mode 100644 modules/web-console/frontend/app/primitives/form-field/password.pug create mode 100644 modules/web-console/frontend/app/primitives/form-field/phone.pug create mode 100644 modules/web-console/frontend/app/primitives/form-field/text.pug create mode 100644 modules/web-console/frontend/app/primitives/form-field/tooltip.pug rename modules/web-console/frontend/app/{modules/states/password.state.js => services/DefaultState.js} (53%) create mode 100644 modules/web-console/frontend/public/images/icons/collapse.svg create mode 100644 modules/web-console/frontend/public/images/icons/exclamation.svg create mode 100644 modules/web-console/frontend/public/images/icons/expand.svg create mode 100644 modules/web-console/frontend/public/images/main-screenshot.png create mode 100644 modules/web-console/frontend/public/images/page-landing-carousel-1.png create mode 100644 modules/web-console/frontend/public/images/page-landing-carousel-2.png create mode 100644 modules/web-console/frontend/public/images/page-landing-carousel-3.png delete mode 100644 modules/web-console/frontend/views/reset.tpl.pug delete mode 100644 modules/web-console/frontend/views/signin.tpl.pug diff --git a/modules/web-console/e2e/testcafe/Dockerfile b/modules/web-console/e2e/testcafe/Dockerfile index e58cbef519750..d890d10fec447 100644 --- a/modules/web-console/e2e/testcafe/Dockerfile +++ b/modules/web-console/e2e/testcafe/Dockerfile @@ -25,6 +25,8 @@ WORKDIR /opt/testcafe/tests COPY . /opt/testcafe/tests +ENV NPM_CONFIG_LOGLEVEL warn + RUN npm install --production && \ npm cache verify --force && \ rm -rf /tmp/* diff --git a/modules/web-console/e2e/testcafe/envtools.js b/modules/web-console/e2e/testcafe/envtools.js index 846a33a725cab..c251b5f094590 100644 --- a/modules/web-console/e2e/testcafe/envtools.js +++ b/modules/web-console/e2e/testcafe/envtools.js @@ -46,6 +46,7 @@ const insertTestUser = ({userId = '000000000000000000000001', token = 'ppw4tPI3J lastName: 'Doe', company: 'TestCompany', country: 'Canada', + industry: 'Banking', admin: true, token, attempts: 0, diff --git a/modules/web-console/e2e/testcafe/fixtures/auth.js b/modules/web-console/e2e/testcafe/fixtures/auth.js index 8f72b8310912e..c84556063dc65 100644 --- a/modules/web-console/e2e/testcafe/fixtures/auth.js +++ b/modules/web-console/e2e/testcafe/fixtures/auth.js @@ -21,7 +21,7 @@ const { AngularJSSelector } = require('testcafe-angular-selectors'); const { removeData, insertTestUser } = require('../envtools'); fixture('Checking Ignite auth screen') - .page `${process.env.APP_URL || 'http://localhost:9001/'}` + .page `${process.env.APP_URL || 'http://localhost:9001/'}signin` .beforeEach(async(t) => { await removeData(); @@ -34,61 +34,59 @@ fixture('Checking Ignite auth screen') test('Testing Ignite signup validation and signup success', async(t) => { async function checkBtnDisabled() { - const btnDisabled = await t.expect(Selector('#signup').getAttribute('disabled')).ok(); + const btnDisabled = await t.expect(Selector('#signup_submit').getAttribute('disabled')).ok(); const btnNotWorks = await t - .click('#signup') + .click('#signup_submit') .expect(Selector('title').innerText).eql('Apache Ignite - Management Tool and Configuration Wizard – Apache Ignite Web Console'); return btnDisabled && btnNotWorks; } - await t.click(Selector('a').withText('Sign Up')); - await t - .click(Selector('#signup_email')) - .typeText(Selector('#signup_email'), 'test@test.com'); + .typeText(AngularJSSelector.byModel('ui_signup.email'), 'test@test.com'); await checkBtnDisabled(); await t - .typeText(AngularJSSelector.byModel('ui.password'), 'qwerty') + .typeText(AngularJSSelector.byModel('ui_signup.password'), 'qwerty') .typeText(AngularJSSelector.byModel('ui_exclude.confirm'), 'qwerty'); await checkBtnDisabled(); await t - .typeText(AngularJSSelector.byModel('ui.firstName'), 'John') - .typeText(AngularJSSelector.byModel('ui.lastName'), 'Doe'); + .typeText(AngularJSSelector.byModel('ui_signup.firstName'), 'John') + .typeText(AngularJSSelector.byModel('ui_signup.lastName'), 'Doe'); await checkBtnDisabled(); await t - .typeText(AngularJSSelector.byModel('ui.company'), 'DevNull LTD'); + .typeText(AngularJSSelector.byModel('ui_signup.company'), 'DevNull LTD'); await checkBtnDisabled(); await t - .click('#country') + .click('#countryInput') .click(Selector('span').withText('Brazil')); - // checking passwords confirm dismatch + // Checking passwords confirm dismatch. await t .click(AngularJSSelector.byModel('ui_exclude.confirm')) .pressKey('ctrl+a delete') .typeText(AngularJSSelector.byModel('ui_exclude.confirm'), 'ytrewq'); await checkBtnDisabled(); + await t .click(AngularJSSelector.byModel('ui_exclude.confirm')) .pressKey('ctrl+a delete') .typeText(AngularJSSelector.byModel('ui_exclude.confirm'), 'qwerty'); - await t.click('#signup') + await t.click('#signup_submit') .expect(Selector('title').innerText).eql('Basic Configuration – Apache Ignite Web Console'); }); test('Testing Ignite validation and successful sign in of existing user', async(t) => { async function checkSignInBtnDisabled() { - const btnDisabled = await t.expect(await Selector('#login').getAttribute('disabled')).ok(); + const btnDisabled = await t.expect(await Selector('#signin_submit').getAttribute('disabled')).ok(); const btnNotWorks = await t - .click('#login') + .click('#signin_submit') .expect(Selector('title').innerText).eql('Apache Ignite - Management Tool and Configuration Wizard – Apache Ignite Web Console'); return btnDisabled && btnNotWorks; @@ -96,14 +94,14 @@ test('Testing Ignite validation and successful sign in of existing user', async( await insertTestUser(); - // checking signin validation + // Checking signin validation. await t .typeText(AngularJSSelector.byModel('ui.email'), 'test@test.com'); await checkSignInBtnDisabled(); await t .typeText(AngularJSSelector.byModel('ui.password'), 'b') - .click('#login'); + .click('#signin_submit'); await t.expect(Selector('#popover-validation-message').withText('Invalid email or password').exists).ok(); await t @@ -112,7 +110,7 @@ test('Testing Ignite validation and successful sign in of existing user', async( .typeText(AngularJSSelector.byModel('ui.email'), 'testtest.com'); await checkSignInBtnDisabled(); - // checking regular sigin in + // Checking regular sigin in await t .click(AngularJSSelector.byModel('ui.email')) .pressKey('ctrl+a delete') @@ -120,7 +118,7 @@ test('Testing Ignite validation and successful sign in of existing user', async( .click(AngularJSSelector.byModel('ui.password')) .pressKey('ctrl+a delete') .typeText(AngularJSSelector.byModel('ui.password'), 'a') - .click('#login') + .click('#signin_submit') .expect(Selector('title').innerText).eql('Basic Configuration – Apache Ignite Web Console'); }); @@ -128,46 +126,43 @@ test('Testing Ignite validation and successful sign in of existing user', async( test('Forbid Ignite signing up of already existing user', async(t) => { await insertTestUser(); - await t.click(Selector('a').withText('Sign Up')); - await t - .click(Selector('#signup_email')) - .typeText(Selector('#signup_email'), 'a@a') - .typeText(AngularJSSelector.byModel('ui.password'), 'a') + .typeText(AngularJSSelector.byModel('ui_signup.email'), 'a@a') + .typeText(AngularJSSelector.byModel('ui_signup.password'), 'a') .typeText(AngularJSSelector.byModel('ui_exclude.confirm'), 'a') - .typeText(AngularJSSelector.byModel('ui.firstName'), 'John') - .typeText(AngularJSSelector.byModel('ui.lastName'), 'Doe') - .typeText(AngularJSSelector.byModel('ui.company'), 'DevNull LTD') - .click('#country') + .typeText(AngularJSSelector.byModel('ui_signup.firstName'), 'John') + .typeText(AngularJSSelector.byModel('ui_signup.lastName'), 'Doe') + .typeText(AngularJSSelector.byModel('ui_signup.company'), 'DevNull LTD') + .click('#countryInput') .click(Selector('span').withText('Brazil')) - .click('#signup'); + .click('#signup_submit'); await t.expect(Selector('#popover-validation-message').withText('A user with the given username is already registered').exists).ok(); }); test('Test Ignite password reset', async(t) => { - await t.click(Selector('#password-forgot-signin')); + await t.click(Selector('#forgot_show')); - // testing incorrect email + // Testing incorrect email. await t - .typeText('#forgot_email', 'testtest') + .typeText('#forgotEmailInput', 'testtest') .expect(await Selector('button').withText('Send it to me').getAttribute('disabled')).ok(); - // testing handling unknown email password reset + // Testing handling unknown email password reset. await t - .click(Selector('#forgot_email')) + .click(Selector('#forgotEmailInput')) .pressKey('ctrl+a delete') - .typeText(Selector('#forgot_email'), 'nonexisting@mail.com') + .typeText(Selector('#forgotEmailInput'), 'nonexisting@mail.com') .click(Selector('button').withText('Send it to me')); await t.expect(Selector('#popover-validation-message').withText('Account with that email address does not exists!').exists).ok(); - // testing regular password reset + // Testing regular password reset. await t - .click(Selector('#forgot_email')) + .click(Selector('#forgotEmailInput')) .pressKey('ctrl+a delete') - .typeText(Selector('#forgot_email'), 'a@a') + .typeText(Selector('#forgotEmailInput'), 'a@a') .click(Selector('button').withText('Send it to me')); await t.expect(Selector('#popover-validation-message').withText('Account with that email address does not exists!').exists).notOk(); diff --git a/modules/web-console/e2e/testcafe/fixtures/user-profile/credentials.js b/modules/web-console/e2e/testcafe/fixtures/user-profile/credentials.js new file mode 100644 index 0000000000000..491a10026d349 --- /dev/null +++ b/modules/web-console/e2e/testcafe/fixtures/user-profile/credentials.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. + */ + +const { Selector } = require('testcafe'); +const { removeData, insertTestUser } = require('../../envtools'); +const { signIn } = require('../../roles'); + +fixture('Checking user credentials change') + .page `${process.env.APP_URL || 'http://localhost:9001/'}settings/profile` + .beforeEach(async(t) => { + await t.setNativeDialogHandler(() => true); + await removeData(); + await insertTestUser(); + await signIn(t); + + await t.navigateTo(`${process.env.APP_URL || 'http://localhost:9001/'}settings/profile`); + }) + .after(async() => { + await removeData(); + }); + +test('Testing secure token change', async(t) => { + await t.click(Selector('header').withAttribute('ng-click', '$ctrl.toggleToken()')); + + const currentToken = await Selector('#current-security-token').innerText; + + await t + .click(Selector('i').withAttribute('ng-click', '$ctrl.generateToken()')) + .expect(Selector('p').withText('Are you sure you want to change security token?').exists) + .ok() + .click('#confirm-btn-ok', {timeout: 5000}); + + await t + .expect(await Selector('#current-security-token').innerText) + .notEql(currentToken); +}); + +test('Testing password change', async(t) => { + await t.click(Selector('header').withAttribute('ng-click', '$ctrl.togglePassword()')); + + await t + .typeText('#passwordInput', 'newPass') + .typeText('#passwordConfirmInput', 'newPass') + .click(Selector('button').withText('Save Changes')); + + await t + .expect(Selector('span').withText('Profile saved.').exists) + .ok(); +}); \ No newline at end of file diff --git a/modules/web-console/e2e/testcafe/fixtures/user-profile.js b/modules/web-console/e2e/testcafe/fixtures/user-profile/profile.js similarity index 61% rename from modules/web-console/e2e/testcafe/fixtures/user-profile.js rename to modules/web-console/e2e/testcafe/fixtures/user-profile/profile.js index 133b37e867fea..738d596bf88f5 100644 --- a/modules/web-console/e2e/testcafe/fixtures/user-profile.js +++ b/modules/web-console/e2e/testcafe/fixtures/user-profile/profile.js @@ -16,8 +16,8 @@ */ const { Selector } = require('testcafe'); -const { removeData, insertTestUser } = require('../envtools'); -const { signIn, signUp } = require('../roles'); +const { removeData, insertTestUser } = require('../../envtools'); +const { signIn } = require('../../roles'); fixture('Checking user profile') .page `${process.env.APP_URL || 'http://localhost:9001/'}settings/profile` @@ -37,38 +37,38 @@ test('Testing user data change', async(t) => { const newUserData = { firstName: { - selector: '#profile-firstname', + selector: '#firstNameInput', value: 'Richard' }, lastName: { - selector: '#profile-lastname', + selector: '#lastNameInput', value: 'Roe' }, email: { - selector: '#profile-email', + selector: '#emailInput', value: 'r.roe@mail.com' }, company: { - selector: '#profile-company', + selector: '#companyInput', value: 'New Company' }, country: { - selector: '#profile-country', + selector: '#countryInput', value: 'Israel' } }; ['firstName', 'lastName', 'email', 'company'].forEach(async(item) => { await t - .click(newUserData[item].selector) - .pressKey('ctrl+a delete') - .typeText(newUserData[item].selector, newUserData[item].value); + .click(newUserData[item].selector) + .pressKey('ctrl+a delete') + .typeText(newUserData[item].selector, newUserData[item].value); }); await t .click(newUserData.country.selector) .click(Selector('span').withText(newUserData.country.value)) - .click(Selector('a').withText('Save')); + .click(Selector('button').withText('Save Changes')); await t.navigateTo(`${process.env.APP_URL || 'http://localhost:9001/'}settings/profile`); @@ -81,33 +81,4 @@ test('Testing user data change', async(t) => { await t .expect(Selector(newUserData.country.selector).innerText) .eql(newUserData.country.value); -}); - -test('Testing secure token change', async(t) => { - await t.click(Selector('a').withAttribute('ng-click', 'toggleToken()')); - - const currentToken = await Selector('#current-security-token').innerText; - - await t - .click(Selector('i').withAttribute('ng-click', 'generateToken()')) - .expect(Selector('p').withText('Are you sure you want to change security token?').exists) - .ok() - .click('#confirm-btn-ok', {timeout: 5000}); - - await t - .expect(await Selector('#current-security-token').innerText) - .notEql(currentToken); -}); - -test('Testing password change', async(t) => { - await t.click(Selector('a').withAttribute('ng-click', 'togglePassword()')); - - await t - .typeText('#profile_password', 'newPass') - .typeText('#profile_confirm', 'newPass') - .click(Selector('a').withText('Save')); - - await t - .expect(Selector('span').withText('Profile saved.').exists) - .ok(); -}); +}); \ No newline at end of file diff --git a/modules/web-console/e2e/testcafe/roles.js b/modules/web-console/e2e/testcafe/roles.js index bfa9f13114867..568a520fec0e1 100644 --- a/modules/web-console/e2e/testcafe/roles.js +++ b/modules/web-console/e2e/testcafe/roles.js @@ -20,21 +20,18 @@ const { Selector } = require('testcafe'); const { AngularJSSelector } = require('testcafe-angular-selectors'); const igniteSignUp = async(t) => { - await t.navigateTo(`${process.env.APP_URL || 'http://localhost:9001/'}`); - - await t.click(Selector('a').withText('Sign Up')); + await t.navigateTo(`${process.env.APP_URL || 'http://localhost:9001/'}signin`); await t - .click(Selector('#signup_email')) - .typeText(Selector('#signup_email'), 'a@a') - .typeText(AngularJSSelector.byModel('ui.password'), 'a') + .typeText(AngularJSSelector.byModel('ui_signup.email'), 'a@a') + .typeText(AngularJSSelector.byModel('ui_signup.password'), 'a') .typeText(AngularJSSelector.byModel('ui_exclude.confirm'), 'a') - .typeText(AngularJSSelector.byModel('ui.firstName'), 'John') - .typeText(AngularJSSelector.byModel('ui.lastName'), 'Doe') - .typeText(AngularJSSelector.byModel('ui.company'), 'DevNull LTD') - .click('#country') + .typeText(AngularJSSelector.byModel('ui_signup.firstName'), 'John') + .typeText(AngularJSSelector.byModel('ui_signup.lastName'), 'Doe') + .typeText(AngularJSSelector.byModel('ui_signup.company'), 'DevNull LTD') + .click('#countryInput') .click(Selector('span').withText('Brazil')) - .click('#signup'); + .click('#signup_submit'); // close modal window await t.click('.modal-header button.close'); @@ -42,12 +39,12 @@ const igniteSignUp = async(t) => { const igniteSignIn = async(t) => { - await t.navigateTo(`${process.env.APP_URL || 'http://localhost:9001/'}`); + await t.navigateTo(`${process.env.APP_URL || 'http://localhost:9001/'}signin`); await t .typeText(AngularJSSelector.byModel('ui.email'), 'a@a') .typeText(AngularJSSelector.byModel('ui.password'), 'a') - .click('#login'); + .click('#signin_submit'); // close modal window await t.click('.modal-header button.close'); diff --git a/modules/web-console/e2e/testcafe/testcafe.js b/modules/web-console/e2e/testcafe/testcafe.js index 597d29a5f985a..66610e8b6a4a3 100644 --- a/modules/web-console/e2e/testcafe/testcafe.js +++ b/modules/web-console/e2e/testcafe/testcafe.js @@ -33,11 +33,11 @@ const BROWSERS = ['chromium:headless --no-sandbox']; // For example: ['chrome', let testcafe = null; const resolveFixturesPaths = () => { - let fixturesPaths = glob.sync('./fixtures/*.js'); + let fixturesPaths = glob.sync(' ./fixtures/**/*.js'); if (process.env.IGNITE_MODULES) { const igniteModulesTestcafe = path.join(process.env.IGNITE_MODULES, 'e2e/testcafe'); - const additionalFixturesPaths = glob.sync(path.join(igniteModulesTestcafe, 'fixtures', '*.js')); + const additionalFixturesPaths = glob.sync(path.join(igniteModulesTestcafe, 'fixtures', '**/*.js')); const relativePaths = new Set(additionalFixturesPaths.map((fixturePath) => path.relative(igniteModulesTestcafe, fixturePath))); fixturesPaths = fixturesPaths.filter((fixturePath) => !relativePaths.has(path.relative(process.cwd(), fixturePath))).concat(additionalFixturesPaths); diff --git a/modules/web-console/e2e/testenv/Dockerfile b/modules/web-console/e2e/testenv/Dockerfile index 5a8f24b904a06..0db2f29294dd6 100644 --- a/modules/web-console/e2e/testenv/Dockerfile +++ b/modules/web-console/e2e/testenv/Dockerfile @@ -49,6 +49,8 @@ RUN curl -SLO "https://nodejs.org/dist/v$NODE_VERSION/node-v$NODE_VERSION-linux- 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 +ENV NPM_CONFIG_LOGLEVEL warn + # Install global node packages. RUN npm install -g pm2 diff --git a/modules/web-console/frontend/app/app.js b/modules/web-console/frontend/app/app.js index c19018c7020ca..6524450716e5c 100644 --- a/modules/web-console/frontend/app/app.js +++ b/modules/web-console/frontend/app/app.js @@ -25,9 +25,7 @@ import './modules/agent/agent.module'; import './modules/nodes/nodes.module'; import './modules/demo/Demo.module'; -import './modules/states/signin.state'; import './modules/states/logout.state'; -import './modules/states/password.state'; import './modules/states/configuration.state'; import './modules/states/admin.state'; import './modules/states/errors.state'; @@ -133,8 +131,12 @@ import uiGridFilters from './components/ui-grid-filters'; import listEditable from './components/list-editable'; import clusterSelector from './components/cluster-selector'; import connectedClusters from './components/connected-clusters'; +import pageSignIn from './components/page-signin'; +import pageLanding from './components/page-landing'; import pageProfile from './components/page-profile'; +import pagePasswordChanged from './components/page-password-changed'; +import pagePasswordReset from './components/page-password-reset'; import igniteServices from './services'; @@ -148,6 +150,7 @@ angular.module('ignite-console', [ // Optional AngularJS modules. 'ngAnimate', 'ngSanitize', + 'ngMessages', // Third party libs. 'btford.socket-io', 'dndLists', @@ -166,6 +169,7 @@ angular.module('ignite-console', [ 'ui.grid.selection', 'ui.router', 'ui.router.state.events', + 'ui.carousel', // Base modules. 'ignite-console.core', 'ignite-console.ace', @@ -178,9 +182,7 @@ angular.module('ignite-console', [ 'ignite-console.nodes', 'ignite-console.demo', // States. - 'ignite-console.states.login', 'ignite-console.states.logout', - 'ignite-console.states.password', 'ignite-console.states.configuration', 'ignite-console.states.admin', 'ignite-console.states.errors', @@ -218,6 +220,10 @@ angular.module('ignite-console', [ connectedClusters.name, igniteListOfRegisteredUsers.name, pageProfile.name, + pageSignIn.name, + pageLanding.name, + pagePasswordChanged.name, + pagePasswordReset.name, // Ignite modules. IgniteModules.name ]) diff --git a/modules/web-console/frontend/app/components/page-configure/index.js b/modules/web-console/frontend/app/components/page-configure/index.js index 172609f0ea203..8c4e3c2a0ab11 100644 --- a/modules/web-console/frontend/app/components/page-configure/index.js +++ b/modules/web-console/frontend/app/components/page-configure/index.js @@ -23,6 +23,9 @@ import ConfigurationDownload from './services/ConfigurationDownload'; export default angular .module('ignite-console.page-configure', []) + .config(['DefaultStateProvider', (DefaultState) => { + DefaultState.setRedirectTo(() => 'base.configuration.tabs'); + }]) .component('pageConfigure', component) .service('PageConfigure', PageConfigure) .service('ConfigureState', ConfigureState) diff --git a/modules/web-console/frontend/app/components/page-landing/index.js b/modules/web-console/frontend/app/components/page-landing/index.js new file mode 100644 index 0000000000000..037f3d451d02c --- /dev/null +++ b/modules/web-console/frontend/app/components/page-landing/index.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. + */ + +import angular from 'angular'; + +import template from './template.pug'; +import './style.scss'; + +export default angular + .module('ignite-console.landing', [ + 'ui.router', + 'ignite-console.user' + ]) + .component('pageLanding', { + template, + controller: class LandingController { + static images = [ + '/images/page-landing-carousel-1.png', + '/images/page-landing-carousel-2.png', + '/images/page-landing-carousel-3.png' + ]; + + $onInit() { + this.images = LandingController.images; + } + } + }) + .config(['$stateProvider', function($stateProvider) { + // set up the states + $stateProvider + .state('landing', { + url: '/', + template: '', + redirectTo: (trans) => { + return trans.injector().get('User').read() + .then(() => { + try { + const {name, params} = JSON.parse(localStorage.getItem('lastStateChangeSuccess')); + + const restored = trans.router.stateService.target(name, params); + + return restored.valid() ? restored : 'base.configuration.tabs'; + } catch (ignored) { + return 'base.configuration.tabs'; + } + }) + .catch(() => true); + }, + unsaved: true + }); + }]); diff --git a/modules/web-console/frontend/app/components/page-landing/style.scss b/modules/web-console/frontend/app/components/page-landing/style.scss new file mode 100644 index 0000000000000..a92206ba27ec6 --- /dev/null +++ b/modules/web-console/frontend/app/components/page-landing/style.scss @@ -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. + */ + +@mixin custom_btn { + font-weight: 500; + padding: 10px 25px !important; +} + +#signin_show { + @include custom_btn; +} + +page-landing { + font-family: Roboto; + + .btn-custom { + @include custom_btn; + } + + section.intro-container-wrapper { + padding: 40px 0; + + background-color: #f9f9f9; + border-bottom: 1px solid #aaaaaa; + + .intro-content { + padding-right: 70px; + } + + h1 { + margin-top: 45px; + font-size: 48px; + line-height: 55px; + font-weight: 300; + } + + h2 { + margin-bottom: 20px; + font-size: 24px; + font-style: italic; + font-weight: 300; + } + + p { + font-size: 16px; + font-weight: 300; + color: #777777; + } + + .btn-custom { + margin-top: 20px; + padding: 10px 40px !important; + border-color: #f9f9f9 !important; + } + } + + section.features-container-wrapper { + padding: 25px 0 60px; + background-color: #ffffff; + + .section-title { + font-size: 38px; + font-weight: 300; + color: #444444; + margin: 30px 0 60px; + } + + .feature { + margin: 30px 0; + + h3 { + font-size: 24px; + font-weight: normal; + color: #000000; + line-height: 28px; + margin-bottom: 10px; + } + + p { + color: #666666; + font-size: 16px; + } + + i.fa { + font-size: 48px; + color: #bbbbbb; + } + } + } +} diff --git a/modules/web-console/frontend/app/components/page-landing/template.pug b/modules/web-console/frontend/app/components/page-landing/template.pug new file mode 100644 index 0000000000000..a1ce4f71d614b --- /dev/null +++ b/modules/web-console/frontend/app/components/page-landing/template.pug @@ -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. + +web-console-header + web-console-header-left + ignite-header-title + web-console-header-right + a#signin_show.btn-ignite.btn-ignite--primary(ui-sref='signin') Sign In + +section.intro-container-wrapper + .container + .col-lg-6.col-md-6.col-sm-6.col-xs-12.intro-content + h1 Web Console + h2 An Interactive Configuration Wizard and Management Tool for Apache™ Ignite® + p It provides an interactive configuration wizard which helps you create and download configuration files and code snippets for your Apache Ignite projects. Additionally, the tool allows you to automatically load SQL metadata from any RDBMS, run SQL queries on your in-memory cache as well as view execution plans, in-memory schema, and streaming charts. + a#signup_show.btn.btn-lg.btn-primary.btn-custom(ui-sref='signin') Try Now + .col-lg-6.col-md-6.col-sm-6.col-xs-12 + ui-carousel(slides='$ctrl.images' autoplay='true' slides-to-show='1' arrows='false') + carousel-item + img(ng-src='{{item}}') +section.features-container-wrapper + .container.features-container + .section-title Web Console allows you to: + .row + .col-lg-6.col-md-6.col-sm-6.col-xs-12.feature + .col-lg-2.col-md-2.col-sm-2.col-xs-2 + i.fa.fa-sitemap + .col-lg-9.col-md-9.col-sm-9.col-xs-9 + h3 Configure Apache Ignite Clusters and Caches + p The Web Console configuration wizard takes you through a step-by-step process to define all of your required configuration parameters. The system then generates a ready-to-use project with all of the required config files for you. + .col-lg-6.col-md-6.col-sm-6.col-xs-12.feature + .col-lg-2.col-md-2.col-sm-2.col-xs-2 + i.fa.fa-search + .col-lg-9.col-md-9.col-sm-9.col-xs-9 + h3 Run Free Form SQL Queries on #[br] Apache Ignite Caches + p By connecting The Web Console to your Apache Ignite cluster, you can execute SQL queries on your in-memory cache. You can also view the execution plan, in-memory schema, and streaming charts for your cluster. + .row + .col-lg-6.col-md-6.col-sm-6.col-xs-12.feature + .col-lg-2.col-md-2.col-sm-2.col-xs-2 + i.fa.fa-database + .col-lg-9.col-md-9.col-sm-9.col-xs-9 + h3 Import Database Schemas from #[br] Virtually Any RDBMS + p To speed the creation of your configuration files, The Web Console allows you to automatically import the database schema from any current RDBMS. and Apache Ignite support virtually any RDBMS including Oracle, SAP, MySQL, PostgreSQL and many more. + .col-lg-6.col-md-6.col-sm-6.col-xs-12.feature + .col-lg-2.col-md-2.col-sm-2.col-xs-2 + i.fa.fa-gears + .col-lg-9.col-md-9.col-sm-9.col-xs-9 + h3 Manage The Web Console users + p With The Web Console you can have accounts with different roles. + .align-center.text-center + a.btn.btn-lg.btn-primary.btn-custom(ui-sref='signin') Get Started + +web-console-footer diff --git a/modules/web-console/frontend/app/components/page-password-changed/controller.js b/modules/web-console/frontend/app/components/page-password-changed/controller.js new file mode 100644 index 0000000000000..5b8301d20771d --- /dev/null +++ b/modules/web-console/frontend/app/components/page-password-changed/controller.js @@ -0,0 +1,18 @@ +/* + * Copyright (C) GridGain Systems. All Rights Reserved. + * _________ _____ __________________ _____ + * __ ____/___________(_)______ /__ ____/______ ____(_)_______ + * _ / __ __ ___/__ / _ __ / _ / __ _ __ `/__ / __ __ \ + * / /_/ / _ / _ / / /_/ / / /_/ / / /_/ / _ / _ / / / + * \____/ /_/ /_/ \_,__/ \____/ \__,_/ /_/ /_/ /_/ + */ + +export default class { + static $inject = ['$state', '$timeout']; + + constructor($state, $timeout) { + $timeout(() => { + $state.go('signin'); + }, 10000); + } +} diff --git a/modules/web-console/frontend/app/components/page-password-changed/index.js b/modules/web-console/frontend/app/components/page-password-changed/index.js new file mode 100644 index 0000000000000..4d624a73e8c36 --- /dev/null +++ b/modules/web-console/frontend/app/components/page-password-changed/index.js @@ -0,0 +1,33 @@ +/* + * Copyright (C) GridGain Systems. All Rights Reserved. + * _________ _____ __________________ _____ + * __ ____/___________(_)______ /__ ____/______ ____(_)_______ + * _ / __ __ ___/__ / _ __ / _ / __ _ __ `/__ / __ __ \ + * / /_/ / _ / _ / / /_/ / / /_/ / / /_/ / _ / _ / / / + * \____/ /_/ /_/ \_,__/ \____/ \__,_/ /_/ /_/ /_/ + */ + +import angular from 'angular'; + +import template from './template.pug'; +import controller from './controller'; + +import './style.scss'; + +export default angular + .module('ignite-console.page-password-changed', [ + ]) + .component('pagePasswordChanged', { + template, + controller + }) + .config(['$stateProvider', ($stateProvider) => { + $stateProvider.state('password.send', { + url: '/changed', + component: 'pagePasswordChanged', + tfMetaTags: { + title: 'Password send' + }, + unsaved: true + }); + }]); diff --git a/modules/web-console/frontend/app/components/page-password-changed/style.scss b/modules/web-console/frontend/app/components/page-password-changed/style.scss new file mode 100644 index 0000000000000..944fc47534340 --- /dev/null +++ b/modules/web-console/frontend/app/components/page-password-changed/style.scss @@ -0,0 +1,34 @@ +/* + * Copyright (C) GridGain Systems. All Rights Reserved. + * _________ _____ __________________ _____ + * __ ____/___________(_)______ /__ ____/______ ____(_)_______ + * _ / __ __ ___/__ / _ __ / _ / __ _ __ `/__ / __ __ \ + * / /_/ / _ / _ / / /_/ / / /_/ / / /_/ / _ / _ / / / + * \____/ /_/ /_/ \_,__/ \____/ \__,_/ /_/ /_/ /_/ + */ + +page-password-changed { + display: flex; + flex: 1 0 auto; + flex-direction: column; + + min-height: 100%; + + font-family: Roboto; + + .body-container { + display: flex; + flex-direction: column; + justify-content: center; + align-items: center; + } + + h2 { + margin-bottom: 30px; + } + + p { + font-size: 16px; + text-align: center; + } +} diff --git a/modules/web-console/frontend/app/components/page-password-changed/template.pug b/modules/web-console/frontend/app/components/page-password-changed/template.pug new file mode 100644 index 0000000000000..33f20d4f41bbe --- /dev/null +++ b/modules/web-console/frontend/app/components/page-password-changed/template.pug @@ -0,0 +1,14 @@ +//- + Copyright (C) GridGain Systems. All Rights Reserved. + _________ _____ __________________ _____ + __ ____/___________(_)______ /__ ____/______ ____(_)_______ + _ / __ __ ___/__ / _ __ / _ / __ _ __ `/__ / __ __ \ + / /_/ / _ / _ / / /_/ / / /_/ / / /_/ / _ / _ / / / + \____/ /_/ /_/ \_,__/ \____/ \__,_/ /_/ /_/ /_/ + +.container.body-container + h2 Ready! + + p + | Further instructions for password reset have been sent to your e-mail address.#[br] + | You'll be redirected back automatically in a few seconds. If not, please #[a(ui-sref='signin') click here]. diff --git a/modules/web-console/frontend/app/components/page-password-reset/controller.js b/modules/web-console/frontend/app/components/page-password-reset/controller.js new file mode 100644 index 0000000000000..6b402282e31b5 --- /dev/null +++ b/modules/web-console/frontend/app/components/page-password-reset/controller.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. + */ + +export default class { + static $inject = ['$modal', '$http', '$state', 'IgniteMessages']; + + /** + * @param {mgcrea.ngStrap.modal.IModalService} $modal + * @param $http + * @param {StateProvider} $state + * @param Messages + */ + constructor($modal, $http, $state, Messages) { + this.$http = $http; + this.$state = $state; + this.Messages = Messages; + } + + $onInit() { + this.$http.post('/api/v1/password/validate/token', {token: this.$state.params.token}) + .then(({data}) => this.ui = data); + } + + // Try to reset user password for provided token. + resetPassword() { + this.$http.post('/api/v1/password/reset', {token: this.ui.token, password: this.ui.password}) + .then(() => { + this.$state.go('signin'); + + this.Messages.showInfo('Password successfully changed'); + }) + .catch(({data, state}) => { + if (state === 503) + this.$state.go('signin'); + + this.Messages.showError(data); + }); + } +} diff --git a/modules/web-console/frontend/app/components/page-password-reset/index.js b/modules/web-console/frontend/app/components/page-password-reset/index.js new file mode 100644 index 0000000000000..e1042a6565c56 --- /dev/null +++ b/modules/web-console/frontend/app/components/page-password-reset/index.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 angular from 'angular'; + +import _ from 'lodash'; + +import template from './template.pug'; +import controller from './controller'; + +import './style.scss'; + +export default angular + .module('ignite-console.page-password-reset', [ + ]) + .component('pagePasswordReset', { + template, + controller + }) + .config(['$stateProvider', ($stateProvider) => { + // set up the states + $stateProvider + .state('password', { + url: '/password', + abstract: true, + template: '' + }) + .state('password.reset', { + url: '/reset?{token}', + component: 'pagePasswordReset', + redirectTo: (trans) => { + if (_.isEmpty(trans.params('to').token)) + return 'signin'; + + return true; + }, + unsaved: true, + tfMetaTags: { + title: 'Reset password' + } + }); + }]); diff --git a/modules/web-console/frontend/app/components/page-password-reset/style.scss b/modules/web-console/frontend/app/components/page-password-reset/style.scss new file mode 100644 index 0000000000000..75f758e676048 --- /dev/null +++ b/modules/web-console/frontend/app/components/page-password-reset/style.scss @@ -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. + */ + +page-password-reset { + display: flex; + flex: 1 0 auto; + flex-direction: column; + + footer { + display: flex; + justify-content: flex-end; + } + + .btn-ignite + .btn-ignite { + margin-left: 10px; + } +} diff --git a/modules/web-console/frontend/app/components/page-password-reset/template.pug b/modules/web-console/frontend/app/components/page-password-reset/template.pug new file mode 100644 index 0000000000000..1acfab82eb78f --- /dev/null +++ b/modules/web-console/frontend/app/components/page-password-reset/template.pug @@ -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. + +include /app/helpers/jade/mixins + +web-console-header + web-console-header-left + ignite-header-title + +.container.body-container + .row + .main-content(ng-if='error') + .text-center + p {{::$ctrl.ui.error}} + div(ng-if='$ctrl.ui.token && !$ctrl.ui.error') + header.header-with-selector + div + h1 Reset Password + + -var form = '$ctrl.form' + form.theme--ignite(name='$ctrl.form' ng-init='reset_info.token = token') + .row + .col-50 + +form-field__email({ + label: 'E-mail:', + model: '$ctrl.ui.email', + disabled: true + }) + + .row + .col-50 + +form-field__password({ + label: 'New password:', + model: '$ctrl.ui.password', + name: '"password"', + required: true, + placeholder: 'New password' + })( + ignite-auto-focus + ignite-on-enter-focus-move='passwordConfirmInput' + ) + + .row + .col-50 + +form-field__password({ + label: 'Confirm password:', + model: 'confirm', + name: '"passwordConfirm"', + required: true, + placeholder: 'Confirm new password' + })( + ignite-on-enter-focus-move='resetForm.$valid && resetPassword(user_info)' + ignite-match='$ctrl.ui.password' + ) + + hr + + footer + a.btn-ignite.btn-ignite--link-success(type='button' ui-sref='default-state') Cancel + button.btn-ignite.btn-ignite--success( + ng-disabled='$ctrl.form.$invalid' + ng-click='$ctrl.resetPassword()' + ) + svg.icon-left(ignite-icon='checkmark') + | Save Changes + +web-console-footer diff --git a/modules/web-console/frontend/app/components/page-profile/controller.js b/modules/web-console/frontend/app/components/page-profile/controller.js index b60340b74d757..3fc7318a01fc8 100644 --- a/modules/web-console/frontend/app/components/page-profile/controller.js +++ b/modules/web-console/frontend/app/components/page-profile/controller.js @@ -25,14 +25,12 @@ export default class PageProfileController { } $onInit() { - const self = this; - - self.ui = {}; + this.ui = {}; this.User.read() - .then((user) => self.ui.user = angular.copy(user)); + .then((user) => this.ui.user = angular.copy(user)); - self.ui.countries = this.Countries.getAll(); + this.ui.countries = this.Countries.getAll(); } toggleToken() { diff --git a/modules/web-console/frontend/app/components/page-profile/index.js b/modules/web-console/frontend/app/components/page-profile/index.js index d9921b3083836..faa3c210823de 100644 --- a/modules/web-console/frontend/app/components/page-profile/index.js +++ b/modules/web-console/frontend/app/components/page-profile/index.js @@ -16,7 +16,11 @@ */ import angular from 'angular'; + import component from './component'; +import template from 'views/base2.pug'; + +import './style.scss'; export default angular .module('ignite-console.page-profile', [ @@ -26,7 +30,14 @@ export default angular // set up the states $stateProvider.state('base.settings.profile', { url: '/profile', - component: 'pageProfile', + views: { + '@': { + template + }, + '@base.settings.profile': { + component: 'pageProfile' + } + }, permission: 'profile', tfMetaTags: { title: 'User profile' diff --git a/modules/web-console/frontend/app/components/page-profile/style.scss b/modules/web-console/frontend/app/components/page-profile/style.scss new file mode 100644 index 0000000000000..a96914ec3ca7e --- /dev/null +++ b/modules/web-console/frontend/app/components/page-profile/style.scss @@ -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. + */ + +page-profile { + .panel--ignite { + width: 100%; + } + + footer { + display: flex; + justify-content: flex-end; + } + + .btn-ignite + .btn-ignite { + margin-left: 10px; + } +} diff --git a/modules/web-console/frontend/app/components/page-profile/template.pug b/modules/web-console/frontend/app/components/page-profile/template.pug index 5331670ddaf98..48aa151127f25 100644 --- a/modules/web-console/frontend/app/components/page-profile/template.pug +++ b/modules/web-console/frontend/app/components/page-profile/template.pug @@ -14,70 +14,143 @@ See the License for the specific language governing permissions and limitations under the License. -mixin lbl(txt) - label.col-sm-2.required.labelFormField #{txt} +include /app/helpers/jade/mixins -mixin lbl-not-required(txt) - label.col-sm-2.labelFormField #{txt} - -.row - .docs-content - .docs-header +div + header.header-with-selector + div h1 User profile - hr - .docs-body - form.form-horizontal(name='$ctrl.form' novalidate) - .col-sm-10(style='padding: 0') - .details-row - +lbl('First name:') - .col-xs-5.col-sm-4 - input#profile-firstname.form-control(ignite-on-enter-focus-move='profile-lastname' type='text' ng-model='$ctrl.ui.user.firstName' placeholder='Input first name' required ignite-auto-focus) - .details-row - +lbl('Last name:') - .col-xs-5.col-sm-4 - input#profile-lastname.form-control(ignite-on-enter-focus-move='profile-email' type='text' ng-model='$ctrl.ui.user.lastName' placeholder='Input last name' required) - .details-row - +lbl('Email:') - .col-xs-5.col-sm-4 - input#profile-email.form-control(ignite-on-enter-focus-move='profile-company' type='email' ng-model='$ctrl.ui.user.email' placeholder='Input email' required) - .details-row - +lbl-not-required('Phone:') - .col-xs-5.col-sm-4 - input#profile-phone.form-control(ignite-on-enter-focus-move='profile-company' type='tel' ng-model='$ctrl.ui.user.phone' placeholder='Input phone') - .details-row - +lbl('Company:') - .col-xs-5.col-sm-4 - input#profile-company.form-control(ignite-on-enter-focus-move='profile-country' type='text' ng-model='$ctrl.ui.user.company' placeholder='Input company name' required) - .details-row - +lbl('Country:') - .col-xs-5.col-sm-4 - button#profile-country.select-toggle.form-control(bs-select bs-options='item.name as item.name for item in $ctrl.ui.countries' type='text' ng-model='$ctrl.ui.user.country' placeholder='Choose your country' ng-required='true') - .details-row#security-token-section - .advanced-options - i.fa( - ng-click='$ctrl.toggleToken()' - ng-class='$ctrl.ui.expandedToken ? "fa-chevron-circle-down" : "fa-chevron-circle-right"') - a(ng-click='$ctrl.toggleToken()') {{$ctrl.ui.expandedToken ? 'Cancel security token changing...' : 'Show security token...'}} - div(ng-if='$ctrl.ui.expandedToken') - +lbl('Security token:') - label#current-security-token {{$ctrl.ui.user.token || 'No security token. Regenerate please.'}} - i.tipLabel.fa.fa-refresh(ng-click='$ctrl.generateToken()' bs-tooltip='' data-title='Generate random security token') - i.tipLabel.fa.fa-clipboard(ignite-copy-to-clipboard='{{$ctrl.ui.user.token}}' bs-tooltip='' data-title='Copy security token to clipboard') - i.tipLabel.icon-help(ng-if=lines bs-tooltip='' data-title='The security token is used for authorization of web agent') - .details-row#change-password-section - .advanced-options - i.fa( - ng-click='$ctrl.togglePassword()' - ng-class='$ctrl.ui.expandedPassword ? "fa-chevron-circle-down" : "fa-chevron-circle-right"') - a(ng-click='$ctrl.togglePassword()') {{$ctrl.ui.expandedPassword ? 'Cancel password changing...' : 'Change password...'}} - div(ng-if='$ctrl.ui.expandedPassword') - .details-row - +lbl('New password:') - .col-xs-5.col-sm-4 - input#profile_password.form-control(ignite-on-enter-focus-move='profile_confirm' type='password' ng-model='$ctrl.ui.user.password' placeholder='New password' required) - .details-row - +lbl('Confirm:') - .col-xs-5.col-sm-4 - input#profile_confirm.form-control(type='password' ng-model='user.confirm' ignite-match='$ctrl.ui.user.password' placeholder='Confirm new password' required) - .col-xs-12.col-sm-12.details-row - button.btn.btn-primary(ng-click='$ctrl.saveUser()' ng-disabled='$ctrl.form.$invalid') Save + + -var form = '$ctrl.form' + form.theme--ignite(name='$ctrl.form' novalidate) + .row + .col-25 + +form-field__text({ + label: 'First name:', + model: '$ctrl.ui.user.firstName', + name: '"firstName"', + required: true, + placeholder: 'Input first name' + })( + ignite-auto-focus + ignite-on-enter-focus-move='lastNameInput' + ) + .col-25 + +form-field__text({ + label: 'Last name:', + model: '$ctrl.ui.user.lastName', + name: '"lastName"', + required: true, + placeholder: 'Input last name' + })( + ignite-on-enter-focus-move='emailInput' + ) + .row + .col-50 + +form-field__email({ + label: 'Email:', + model: '$ctrl.ui.user.email', + name: '"email"', + required: true, + placeholder: 'Input email' + })( + ignite-on-enter-focus-move='phoneInput' + ) + .row + .col-25 + +form-field__phone({ + label: 'Phone:', + model: '$ctrl.ui.user.phone', + name: '"phone"', + optional: true, + placeholder: 'Input phone (ex.: +15417543010)' + })( + ignite-on-enter-focus-move='companyInput' + ) + .col-25 + +form-field__dropdown({ + label: 'Country:', + model: '$ctrl.ui.user.country', + name: '"country"', + required: true, + placeholder: 'Choose your country', + options: '$ctrl.ui.countries' + }) + .row + .col-50 + +form-field__text({ + label: 'Company:', + model: '$ctrl.ui.user.company', + name: '"company"', + required: true, + placeholder: 'Input company name' + })( + ignite-on-enter-focus-move='countryInput' + ) + + .row#security-token-section + .col-50 + .panel--ignite.panel--collapse(ng-class='{ in: !$ctrl.ui.expandedToken }') + header(ng-click='$ctrl.toggleToken()') + svg(ignite-icon='expand') + svg(ignite-icon='collapse') + + | {{$ctrl.ui.expandedToken ? 'Cancel security token changing...' : 'Show security token...'}} + hr + section(ng-if='$ctrl.ui.expandedToken') + .row + .col-25 + label.required Security token: + .col-75 + label#current-security-token {{$ctrl.ui.user.token || 'No security token. Regenerate please.'}} + label + i.tipLabel.fa.fa-refresh(ng-click='$ctrl.generateToken()' bs-tooltip='' data-title='Generate random security token') + i.tipLabel.fa.fa-clipboard(ignite-copy-to-clipboard='{{$ctrl.ui.user.token}}' bs-tooltip='' data-title='Copy security token to clipboard') + i.tipLabel.icon-help(ng-if=lines bs-tooltip='' data-title='The security token is used for authorization of web agent') + + .row + .col-50 + .panel--ignite.panel--collapse(ng-class='{ in: !$ctrl.ui.expandedPassword }') + header(ng-click='$ctrl.togglePassword()') + svg(ignite-icon='expand') + svg(ignite-icon='collapse') + + | {{ $ctrl.ui.expandedPassword ? 'Cancel password changing...' : 'Change password...' }} + hr + section(ng-if='$ctrl.ui.expandedPassword') + .row + .col-100 + +form-field__password({ + label: 'New password:', + model: '$ctrl.ui.user.password', + name: '"password"', + required: true, + placeholder: 'New password' + })( + ignite-auto-focus + ignite-on-enter-focus-move='passwordConfirmInput' + ) + + .row + .col-100 + +form-field__password({ + label: 'Confirm password:', + model: 'user.confirm', + name: '"passwordConfirm"', + required: true, + placeholder: 'Confirm new password' + })( + ignite-on-enter-focus-move='passwordConfirmInput' + ignite-match='$ctrl.ui.user.password' + ) + + hr + + footer + a.btn-ignite.btn-ignite--link-success(type='button' ui-sref='default-state') Cancel + button.btn-ignite.btn-ignite--success( + ng-click='$ctrl.saveUser()' + ng-disabled='$ctrl.form.$invalid' + ) + svg.icon-left(ignite-icon='checkmark') + | Save Changes diff --git a/modules/web-console/frontend/app/controllers/auth.controller.js b/modules/web-console/frontend/app/components/page-signin/controller.js similarity index 71% rename from modules/web-console/frontend/app/controllers/auth.controller.js rename to modules/web-console/frontend/app/components/page-signin/controller.js index caa74dc32001c..585bb7ab64355 100644 --- a/modules/web-console/frontend/app/controllers/auth.controller.js +++ b/modules/web-console/frontend/app/components/page-signin/controller.js @@ -16,14 +16,15 @@ */ // Sign in controller. -export default [ - '$scope', '$uiRouterGlobals', 'IgniteFocus', 'IgniteCountries', 'Auth', - ($scope, $uiRouterGlobals, Focus, Countries, Auth) => { - $scope.auth = Auth.auth; - $scope.forgotPassword = Auth.forgotPassword; - $scope.action = 'signin'; - $scope.countries = Countries.getAll(); +export default class { + static $inject = ['$scope', '$uiRouterGlobals', 'IgniteFocus', 'IgniteCountries', 'Auth']; + + constructor($scope, $uiRouterGlobals, Focus, Countries, Auth) { + this.auth = Auth.auth; + this.forgotPassword = Auth.forgotPassword; + this.action = 'signin'; + this.countries = Countries.getAll(); Focus.move('user_email'); } -]; +} diff --git a/modules/web-console/frontend/app/components/page-signin/index.js b/modules/web-console/frontend/app/components/page-signin/index.js new file mode 100644 index 0000000000000..6be374f27b3cc --- /dev/null +++ b/modules/web-console/frontend/app/components/page-signin/index.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. + */ + +import angular from 'angular'; + +import template from './template.pug'; +import controller from './controller'; +import './style.scss'; + +export default angular + .module('ignite-console.sign-in', [ + 'ui.router', + 'ignite-console.user' + ]) + .component('pageSignIn', { + controller, + template + }) + .config(['$stateProvider', function($stateProvider) { + // set up the states + $stateProvider + .state('signin', { + url: '/signin', + template: '', + redirectTo: (trans) => { + return trans.injector().get('User').read() + .then(() => { + try { + const {name, params} = JSON.parse(localStorage.getItem('lastStateChangeSuccess')); + + const restored = trans.router.stateService.target(name, params); + + return restored.valid() ? restored : 'default-state'; + } catch (ignored) { + return 'default-state'; + } + }) + .catch(() => true); + }, + unsaved: true + }); + }]); diff --git a/modules/web-console/frontend/app/components/page-signin/style.scss b/modules/web-console/frontend/app/components/page-signin/style.scss new file mode 100644 index 0000000000000..7e13ffe502efd --- /dev/null +++ b/modules/web-console/frontend/app/components/page-signin/style.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. + */ + +page-sign-in { + display: flex; + flex-direction: column; + flex: 1 0 auto; + + font-family: Roboto; + + h3 { + font-size: 38px; + font-weight: 300; + margin: 30px 0 60px; + } + + section { + flex-grow: 1; + padding: 25px 0 60px; + + background-color: #ffffff; + color: #444444; + } +} \ No newline at end of file diff --git a/modules/web-console/frontend/app/components/page-signin/template.pug b/modules/web-console/frontend/app/components/page-signin/template.pug new file mode 100644 index 0000000000000..ae9a0ca889ad0 --- /dev/null +++ b/modules/web-console/frontend/app/components/page-signin/template.pug @@ -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. + +include /app/helpers/jade/mixins + +web-console-header + web-console-header-left + ignite-header-title + +section + .container + .row + .col-lg-6.col-md-6.col-sm-6.col-xs-12 + .row + .col-xs-12.col-md-11 + -var form = 'form' + form(name='form' novalidate) + .settings-row + h3 Don't Have An Account? + .settings-row + +form-field__email({ + label: 'Email:', + model: 'ui_signup.email', + name: '"signupEmail"', + placeholder: 'Input email', + required: true + })( + ignite-on-enter-focus-move='passwordInput' + ) + .settings-row + +form-field__password({ + label: 'Password:', + model: 'ui_signup.password', + name: '"password"', + placeholder: 'Input password', + required: true + })( + ignite-on-enter-focus-move='confirmInput' + ) + .settings-row + +form-field__password({ + label: 'Confirm:', + model: 'ui_exclude.confirm', + name: '"confirm"', + placeholder: 'Confirm password', + required: true + })( + ignite-on-enter-focus-move='firstNameInput' + ignite-match='ui_signup.password' + ) + .settings-row + +form-field__text({ + label: 'First name:', + model: 'ui_signup.firstName', + name: '"firstName"', + placeholder: 'Input first name', + required: true + })( + ignite-on-enter-focus-move='lastNameInput' + ) + .settings-row + +form-field__text({ + label: 'Last name:', + model: 'ui_signup.lastName', + name: '"lastName"', + placeholder: 'Input last name', + required: true + })( + ignite-on-enter-focus-move='companyInput' + ) + .settings-row + +form-field__text({ + label: 'Company:', + model: 'ui_signup.company', + name: '"company"', + placeholder: 'Input company name', + required: true + })( + ignite-on-enter-focus-move='countryInput' + ) + .settings-row + +form-field__dropdown({ + label: 'Country:', + model: 'ui_signup.country', + name: '"country"', + required: true, + placeholder: 'Choose your country', + options: '$ctrl.countries' + })( + ignite-on-enter-focus-move='signup_submit' + ) + .login-footer + button#signup_submit.btn-ignite.btn-ignite--primary(ng-click='$ctrl.auth("signup", ui_signup)' ng-disabled='form.$invalid') Sign Up + + .col-lg-6.col-md-6.col-sm-6.col-xs-12(ng-init='$ctrl.action = "signin"') + .row + .col-xs-12.col-md-11 + -var form = 'form_signin' + form.row(ng-show='$ctrl.action == "signin"' name='form_signin' novalidate) + .settings-row + h3 Sign In + .settings-row + +form-field__email({ + label: 'Email:', + model: 'ui.email', + name: '"signinEmail"', + placeholder: 'Input email', + required: true + })( + ignite-auto-focus + ignite-on-enter-focus-move='singinPasswordInput' + ) + .settings-row + +form-field__password({ + label: 'Password:', + model: 'ui.password', + name: '"signinPassword"', + placeholder: 'Input password', + required: true + })( + ignite-on-enter='form_signin.$valid && $ctrl.auth($ctrl.action, ui)' + ) + .login-footer + a.labelField#forgot_show(ng-click='$ctrl.action = "password/forgot"' ignite-on-click-focus='forgot_email') Forgot password? + button#signin_submit.btn-ignite.btn-ignite--primary(ng-click='$ctrl.auth("signin", ui)' ng-disabled='form_signin.$invalid') Sign In + + - var form = 'form_forgot' + form.row(ng-show='$ctrl.action == "password/forgot"' name='form_forgot' novalidate) + .settings-row + h3 Forgot password? + .settings-row + p.col-xs-12.col-md-11 That's ok! Simply enter your email below and a reset password link will be sent to you via email. You can then follow that link and select a new password. + .settings-row + +form-field__email({ + label: 'Email:', + model: 'ui.email', + name: '"forgotEmail"', + placeholder: 'Input email', + required: true + })( + ignite-auto-focus + ignite-on-enter='form_forgot.$valid && $ctrl.forgotPassword(ui)' + ) + .login-footer + a.labelField#forgot_signin(ng-click='$ctrl.action = "signin"' ignite-on-click-focus='signin_email') Sign In + button#forgot_submit.btn-ignite.btn-ignite--primary(ng-click='$ctrl.forgotPassword(ui)' ng-disabled='form_forgot.$invalid') Send it to me + +web-console-footer diff --git a/modules/web-console/frontend/app/components/web-console-header/style.scss b/modules/web-console/frontend/app/components/web-console-header/style.scss index db49c3e1b7aaf..71766c95b4ea3 100644 --- a/modules/web-console/frontend/app/components/web-console-header/style.scss +++ b/modules/web-console/frontend/app/components/web-console-header/style.scss @@ -22,6 +22,7 @@ web-console-header { $bottom-border-width: 4px; display: block; + min-height: 85px; font-family: Roboto; font-size: 16px; border-bottom: $bottom-border-width solid red; diff --git a/modules/web-console/frontend/app/components/web-console-header/template.pug b/modules/web-console/frontend/app/components/web-console-header/template.pug index a5adbb161a9fd..a05e5f818b415 100644 --- a/modules/web-console/frontend/app/components/web-console-header/template.pug +++ b/modules/web-console/frontend/app/components/web-console-header/template.pug @@ -26,7 +26,7 @@ .wch-content.container connected-clusters(ng-if='$ctrl.$rootScope.user && !$ctrl.$rootScope.IgniteDemoMode && $ctrl.isConnectedClustersVisible && !$root.user.becomeUsed') - a(ui-sref='signin') + a(ui-sref='landing') img.wch-logo(ng-src='{{::$ctrl.branding.headerLogo}}') .wch-slot.wch-slot-left(ng-transclude='slotLeft') diff --git a/modules/web-console/frontend/app/data/countries.json b/modules/web-console/frontend/app/data/countries.json index 18dbe64869617..b340f02ff6bcd 100644 --- a/modules/web-console/frontend/app/data/countries.json +++ b/modules/web-console/frontend/app/data/countries.json @@ -1,98 +1,122 @@ [ { - "name": "United States", + "label": "United States", + "value": "United States", "code": "USA" }, { - "name": "Canada", + "label": "Canada", + "value": "Canada", "code": "CAN" }, { - "name": "United Kingdom", + "label": "United Kingdom", + "value": "United Kingdom", "code": "GBR" }, { - "name": "Germany", + "label": "Germany", + "value": "Germany", "code": "DEU" }, { - "name": "France", + "label": "France", + "value": "France", "code": "FRA" }, { - "name": "Belgium", + "label": "Belgium", + "value": "Belgium", "code": "BEL" }, { - "name": "Switzerland", + "label": "Switzerland", + "value": "Switzerland", "code": "CHE" }, { - "name": "Netherlands", + "label": "Netherlands", + "value": "Netherlands", "code": "NLD" }, { - "name": "Israel", + "label": "Israel", + "value": "Israel", "code": "ISR" }, { - "name": "Sweden", + "label": "Sweden", + "value": "Sweden", "code": "SWE" }, { - "name": "Russia", + "label": "Russia", + "value": "Russia", "code": "RUS" }, { - "name": "Other Europe", + "label": "Other Europe", + "value": "Other Europe", "code": "Other Europe" }, { - "name": "China", + "label": "China", + "value": "China", "code": "CHN" }, { - "name": "India", + "label": "India", + "value": "India", "code": "IND" }, { - "name": "Japan", + "label": "Japan", + "value": "Japan", "code": "JPN" }, { - "name": "Other Asia", + "label": "Other Asia", + "value": "Other Asia", "code": "Other Asia" }, { - "name": "Australia", + "label": "Australia", + "value": "Australia", "code": "AUS" }, { - "name": "Brazil", + "label": "Brazil", + "value": "Brazil", "code": "BRA" }, { - "name": "Argentina", + "label": "Argentina", + "value": "Argentina", "code": "ARG" }, { - "name": "Other South America", + "label": "Other South America", + "value": "Other South America", "code": "Other South America" }, { - "name": "South Africa", + "label": "South Africa", + "value": "South Africa", "code": "ZAF" }, { - "name": "Nigeria", + "label": "Nigeria", + "value": "Nigeria", "code": "NGA" }, { - "name": "Other Africa", + "label": "Other Africa", + "value": "Other Africa", "code": "Other Africa" }, { - "name": "Rest of the World", + "label": "Rest of the World", + "value": "Rest of the World", "code": "Rest of the World" } ] diff --git a/modules/web-console/frontend/app/helpers/jade/mixins.pug b/modules/web-console/frontend/app/helpers/jade/mixins.pug index 7eb4d8946252d..1d3b161560051 100644 --- a/modules/web-console/frontend/app/helpers/jade/mixins.pug +++ b/modules/web-console/frontend/app/helpers/jade/mixins.pug @@ -22,6 +22,7 @@ include ../../primitives/dropdown/index include ../../primitives/tooltip/index include ../../primitives/radio/index include ../../primitives/switcher/index +include ../../primitives/form-field/index //- Mixin for advanced options toggle. mixin advanced-options-toggle(click, cond, showMessage, hideMessage) 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 8d05a459a4b3a..0ce936cdc870d 100644 --- a/modules/web-console/frontend/app/modules/states/configuration.state.js +++ b/modules/web-console/frontend/app/modules/states/configuration.state.js @@ -71,6 +71,7 @@ angular.module('ignite-console.states.configuration', ['ui.router']) return PageConfigure.onStateEnterRedirect(trans.to()); }, + failState: 'signin', tfMetaTags: { title: 'Configuration' } diff --git a/modules/web-console/frontend/app/modules/states/signin.state.js b/modules/web-console/frontend/app/modules/states/signin.state.js deleted file mode 100644 index 8afcaa3d39ee0..0000000000000 --- a/modules/web-console/frontend/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.tpl.pug'; -import controller from 'app/controllers/auth.controller'; - -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, - redirectTo: (trans) => { - return trans.injector().get('User').read() - .then(() => { - try { - const {name, params} = JSON.parse(localStorage.getItem('lastStateChangeSuccess')); - - const restored = trans.router.stateService.target(name, params); - - return restored.valid() ? restored : 'base.configuration.tabs'; - } catch (ignored) { - return 'base.configuration.tabs'; - } - }) - .catch(() => true); - }, - controller, - controllerAs: '$ctrl', - unsaved: true - }); -}]); 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 10fad5ed301a5..88c844a0a128c 100644 --- a/modules/web-console/frontend/app/modules/user/Auth.service.js +++ b/modules/web-console/frontend/app/modules/user/Auth.service.js @@ -21,7 +21,7 @@ export default ['Auth', ['$http', '$rootScope', '$state', '$window', 'IgniteErro forgotPassword(userInfo) { $http.post('/api/v1/password/forgot', userInfo) .then(() => $state.go('password.send')) - .catch(({data}) => ErrorPopover.show('forgot_email', Messages.errorMessage(null, data))); + .catch(({data}) => ErrorPopover.show('forgotEmailInput', Messages.errorMessage(null, data))); }, auth(action, userInfo) { $http.post('/api/v1/' + action, userInfo) @@ -33,12 +33,12 @@ export default ['Auth', ['$http', '$rootScope', '$state', '$window', 'IgniteErro .then((user) => { $root.$broadcast('user', user); - $state.go('base.configuration.tabs'); + $state.go('default-state'); $root.gettingStarted.tryShow(); }); }) - .catch((res) => ErrorPopover.show(action + '_email', Messages.errorMessage(null, res))); + .catch((res) => ErrorPopover.show(action + 'EmailInput', Messages.errorMessage(null, res))); }, logout() { $http.post('/api/v1/logout') diff --git a/modules/web-console/frontend/app/primitives/form-field/checkbox.pug b/modules/web-console/frontend/app/primitives/form-field/checkbox.pug new file mode 100644 index 0000000000000..09596d46287e2 --- /dev/null +++ b/modules/web-console/frontend/app/primitives/form-field/checkbox.pug @@ -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. + +mixin form-field__checkbox({ label, model, name, disabled, required, tip }) + .form-field.form-field__checkbox + +form-field__label({ label, name, required }) + +form-field__tooltip({ title: tip, options: tipOpts }) + + .form-field__control + - attributes.type='checkbox' + +form-field__input({ name, model, disabled, required, placeholder })(attributes=attributes) + + .form-field__errors( + ng-messages=`(${form}[${name}].$dirty || ${form}[${name}].$touched || ${form}[${name}].$submitted) && ${form}[${name}].$invalid ? ${form}[${name}].$error : {}` + ) + if block + block diff --git a/modules/web-console/frontend/app/primitives/form-field/dropdown.pug b/modules/web-console/frontend/app/primitives/form-field/dropdown.pug new file mode 100644 index 0000000000000..cb058295c0077 --- /dev/null +++ b/modules/web-console/frontend/app/primitives/form-field/dropdown.pug @@ -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__dropdown({ label, model, name, disabled, required, multiple, placeholder, placeholderEmpty, options, tip }) + mixin __form-field__input() + button.select-toggle( + id=`{{ ${name} }}Input` + name=`{{ ${name} }}` + + data-placeholder=placeholderEmpty ? `{{ ${options}.length > 0 ? '${placeholder}' : '${placeholderEmpty}' }}` : placeholder + + ng-model=model + ng-disabled=disabled && `${disabled}` + ng-required=required && `${required}` + + bs-select + bs-options=`item.value as item.label for item in ${options}` + + data-multiple=multiple ? '1' : false + + tabindex='0' + )&attributes(attributes.attributes) + + .form-field(id=`{{ ${name} }}Field`) + +form-field__label({ label, name, required }) + +form-field__tooltip({ title: tip, options: tipOpts }) + + .form-field__control + +__form-field__input(attributes=attributes) + + .form-field__errors( + ng-messages=`(${form}[${name}].$dirty || ${form}[${name}].$touched || ${form}[${name}].$submitted) && ${form}[${name}].$invalid ? ${form}[${name}].$error : {}` + ) + if block + block + + if required + +form-field__error({ name, error: 'required', message: `${errLbl} could not be empty!` }) diff --git a/modules/web-console/frontend/app/primitives/form-field/email.pug b/modules/web-console/frontend/app/primitives/form-field/email.pug new file mode 100644 index 0000000000000..5fccdb29e363a --- /dev/null +++ b/modules/web-console/frontend/app/primitives/form-field/email.pug @@ -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. + +mixin form-field__email({ label, model, name, disabled, required, placeholder, tip }) + -var errLbl = label.substring(0, label.length - 1) + + .form-field + +form-field__label({ label, name, required, disabled }) + +form-field__tooltip({ title: tip, options: tipOpts }) + + .form-field__control + - attributes.type='email' + +form-field__input({ name, model, disabled, required, placeholder })(attributes=attributes) + + .form-field__errors( + ng-messages=`(${form}[${name}].$dirty || ${form}[${name}].$touched || ${form}[${name}].$submitted) && ${form}[${name}].$invalid ? ${form}[${name}].$error : {}` + ) + if block + block + + if required + +form-field__error({ name, error: 'required', message: `${errLbl} could not be empty!` }) + + +form-field__error({ name, error: 'email', message: `${errLbl} has invalid format!` }) diff --git a/modules/web-console/frontend/app/primitives/form-field/error.pug b/modules/web-console/frontend/app/primitives/form-field/error.pug new file mode 100644 index 0000000000000..34b03c2e203c4 --- /dev/null +++ b/modules/web-console/frontend/app/primitives/form-field/error.pug @@ -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__error({ error, message }) + .form-field__error(ng-message=error) + div( + bs-tooltip='' + data-title=message + data-placement='top' + data-template=` +
+
+
+
` + ) + svg(ignite-icon='exclamation') diff --git a/modules/web-console/frontend/app/primitives/form-field/index.pug b/modules/web-console/frontend/app/primitives/form-field/index.pug new file mode 100644 index 0000000000000..9b67d6c8e791e --- /dev/null +++ b/modules/web-console/frontend/app/primitives/form-field/index.pug @@ -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. + +include ./error +include ./label +include ./tooltip +include ./input +include ./text +include ./number +include ./email +include ./password +include ./phone +include ./dropdown +include ./checkbox diff --git a/modules/web-console/frontend/app/primitives/form-field/index.scss b/modules/web-console/frontend/app/primitives/form-field/index.scss index 01dd941c0f9e2..e6c0a58a782be 100644 --- a/modules/web-console/frontend/app/primitives/form-field/index.scss +++ b/modules/web-console/frontend/app/primitives/form-field/index.scss @@ -1,4 +1,3 @@ - /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -89,3 +88,151 @@ } } } + +.form-field { + position: relative; + width: 100%; + font-family: Roboto; + + &__label { + display: flex; + margin: 0 0 4px; + + color: #424242; + font-size: 14px; + line-height: 1.25; + + .icon-help { + line-height: 14px; + } + + &.required:after { + content: ''; + } + + i { + font-style: normal; + color: $gray-light; + } + } + + &__control { + overflow: visible; + display: flex; + width: 100%; + + & > input::placeholder, + & > button.select-toggle.placeholder { + color: rgba(66, 66, 66, 0.5); + text-align: left; + } + + & > input, + & > button:not(.btn-ignite) { + outline: none; + overflow: visible; + + box-sizing: border-box; + width: 100%; + max-width: initial; + height: 36px; + padding: 9px 10px; + margin-right: 0; + + border: solid 1px #c5c5c5; + border-radius: 4px; + background-color: #ffffff; + box-shadow: none; + + color: $text-color; + text-align: left; + line-height: 16px; + + &.ng-invalid:not(.ng-pristine), + &.ng-invalid.ng-touched { + border-color: #c5c5c5; + box-shadow: none; + } + + &.ng-invalid:focus, + &:focus { + border-color: $ignite-brand-success; + box-shadow: none; + } + } + } + + &__errors { + position: absolute; + right: 0; + bottom: 0; + + [ng-message] { + // TODO: remove after replace all fields to new + overflow: visible !important; + animation: none !important; + } + } + + &__error { + z-index: 2; + position: relative; + width: 0; + height: 36px; + float: right; + + color: $brand-primary; + line-height: $input-height; + pointer-events: initial; + text-align: center; + + &:before { + position: absolute; + right: 0; + width: 38px; + } + + div { + z-index: 1; + position: absolute; + top: 0; + right: 0; + width: 36px; + height: 36px; + } + + svg { + position: absolute; + top: 10px; + right: 0; + width: 38px; + } + } + + [disabled] { + opacity: .5; + } +} + +.form-field__checkbox { + display: flex; + + .form-field { + &__label { + order: 1; + margin: 0; + } + + &__control { + width: auto; + margin-right: 10px; + padding: 3px 0; + + input { + width: auto; + height: auto; + margin: 0; + } + } + } +} diff --git a/modules/web-console/frontend/app/primitives/form-field/input.pug b/modules/web-console/frontend/app/primitives/form-field/input.pug new file mode 100644 index 0000000000000..0fee77b33ac97 --- /dev/null +++ b/modules/web-console/frontend/app/primitives/form-field/input.pug @@ -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. + +mixin form-field__input({ name, model, disabled, required, placeholder }) + input( + id=`{{ ${name} }}Input` + name=`{{ ${name} }}` + placeholder=placeholder + + ng-model=model + + ng-required=required && `${required}` + ng-disabled=disabled && `${disabled}` + ng-focus='tableReset()' + )&attributes(attributes ? attributes.attributes ? attributes.attributes : attributes : {}) diff --git a/modules/web-console/frontend/app/primitives/form-field/label.pug b/modules/web-console/frontend/app/primitives/form-field/label.pug new file mode 100644 index 0000000000000..d725f9d65afea --- /dev/null +++ b/modules/web-console/frontend/app/primitives/form-field/label.pug @@ -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. + +mixin form-field__label({ label, name, required, optional, disabled }) + -var colon = label[label.length-1] === ':' ? ':' : ''; + - label = label[label.length-1] === ':' ? label.substring(0, label.length - 1) : label + - optional = optional ? ' (optional)' : ''; + + label.form-field__label( + id=name && `{{ ${name} }}Label` + for=name && `{{ ${name} }}Input` + class=`{{ ${required} ? 'required' : '' }}` + ng-disabled=disabled && `${disabled}` + ) + span !{label}!{optional}!{colon} + if block + block diff --git a/modules/web-console/frontend/app/primitives/form-field/number.pug b/modules/web-console/frontend/app/primitives/form-field/number.pug new file mode 100644 index 0000000000000..e5f14548f969f --- /dev/null +++ b/modules/web-console/frontend/app/primitives/form-field/number.pug @@ -0,0 +1,46 @@ +//- + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT 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__number({ label, model, name, disabled, required, placeholder, tip, min, max, step }) + -var errLbl = label.substring(0, label.length - 1) + + .form-field + +form-field__label({ label, name, required }) + +form-field__tooltip({ title: tip, options: tipOpts }) + + .form-field__control + - attributes.type = 'number' + - attributes.min = min ? min : '0' + - attributes.max = max ? max : '{{ Number.MAX_VALUE }}' + - attributes.step = step ? step : '1' + +form-field__input({ name, model, disabled, required, placeholder })(attributes=attributes) + + .form-field__errors( + ng-messages=`(${form}[${name}].$dirty || ${form}[${name}].$touched || ${form}[${name}].$submitted) && ${form}[${name}].$invalid ? ${form}[${name}].$error : {}` + ) + if block + block + + if required + +form-field__error({ error: 'required', message: `${errLbl} could not be empty!` }) + + if min + +form-field__error({ error: 'min', message: `${errLbl} is less than allowable minimum: ${ min || 0 }`}) + + if max + +form-field__error({ error: 'max', message: `${errLbl} is more than allowable maximum: ${ max }`}) + + +form-field__error({ error: 'number', message: 'Only numbers allowed' }) diff --git a/modules/web-console/frontend/app/primitives/form-field/password.pug b/modules/web-console/frontend/app/primitives/form-field/password.pug new file mode 100644 index 0000000000000..ba38cce8e833d --- /dev/null +++ b/modules/web-console/frontend/app/primitives/form-field/password.pug @@ -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. + +mixin form-field__password({ label, model, name, disabled, required, placeholder, tip }) + -var errLbl = label.substring(0, label.length - 1) + + .form-field + +form-field__label({ label, name, required }) + +form-field__tooltip({ title: tip, options: tipOpts }) + + .form-field__control + - attributes.type='password' + +form-field__input({ name, model, disabled, required, placeholder })(attributes=attributes) + + .form-field__errors( + ng-messages=`(${form}[${name}].$dirty || ${form}[${name}].$touched || ${form}[${name}].$submitted) && ${form}[${name}].$invalid ? ${form}[${name}].$error : {}` + ) + if block + block + + if required + +form-field__error({ name, error: 'required', message: `${errLbl} could not be empty!` }) + + +form-field__error({ name, error: 'mismatch', message: `Password does not match the confirm password!` }) diff --git a/modules/web-console/frontend/app/primitives/form-field/phone.pug b/modules/web-console/frontend/app/primitives/form-field/phone.pug new file mode 100644 index 0000000000000..b65c5d22305c7 --- /dev/null +++ b/modules/web-console/frontend/app/primitives/form-field/phone.pug @@ -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. + +mixin form-field__phone({ label, model, name, disabled, required, optional, placeholder, tip }) + -var errLbl = label.substring(0, label.length - 1) + + .form-field + +form-field__label({ label, name, required, optional }) + +form-field__tooltip({ title: tip, options: tipOpts }) + + .form-field__control + - attributes.type='tel' + +form-field__input({ name, model, disabled, required, placeholder })(attributes=attributes) + + .form-field__errors( + ng-messages=`(${form}[${name}].$dirty || ${form}[${name}].$touched || ${form}[${name}].$submitted) && ${form}[${name}].$invalid ? ${form}[${name}].$error : {}` + ) + if block + block + + if required + +form-field__error({ name, error: 'required', message: `${errLbl} could not be empty!` }) diff --git a/modules/web-console/frontend/app/primitives/form-field/text.pug b/modules/web-console/frontend/app/primitives/form-field/text.pug new file mode 100644 index 0000000000000..8f63f19f2aeaa --- /dev/null +++ b/modules/web-console/frontend/app/primitives/form-field/text.pug @@ -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. + +mixin form-field__text({ label, model, name, disabled, required, placeholder, tip }) + -var errLbl = label.substring(0, label.length - 1) + + .form-field(id=`{{ ${name} }}Field`) + +form-field__label({ label, name, required, disabled }) + +form-field__tooltip({ title: tip, options: tipOpts }) + + .form-field__control + - attributes.type='text' + +form-field__input({ name, model, disabled, required, placeholder })(attributes=attributes) + + .form-field__errors( + ng-messages=`(${form}[${name}].$dirty || ${form}[${name}].$touched || ${form}[${name}].$submitted) && ${form}[${name}].$invalid ? ${form}[${name}].$error : {}` + ) + if block + block + + if required + +form-field__error({ name, error: 'required', message: `${errLbl} could not be empty!` }) diff --git a/modules/web-console/frontend/app/primitives/form-field/tooltip.pug b/modules/web-console/frontend/app/primitives/form-field/tooltip.pug new file mode 100644 index 0000000000000..08ffd83c39f33 --- /dev/null +++ b/modules/web-console/frontend/app/primitives/form-field/tooltip.pug @@ -0,0 +1,18 @@ +//- + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT 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__tooltip({ title, options }) + +tooltip(title, options) diff --git a/modules/web-console/frontend/app/primitives/panel/index.scss b/modules/web-console/frontend/app/primitives/panel/index.scss index 56cecfe57646f..e16210ce5b2b2 100644 --- a/modules/web-console/frontend/app/primitives/panel/index.scss +++ b/modules/web-console/frontend/app/primitives/panel/index.scss @@ -17,35 +17,91 @@ @import '../../../public/stylesheets/variables'; -.panel--ignite { - border: none; - border-radius: 0 0 4px 4px; +.panel { + &--ignite { + border: none; + border-radius: 0 0 4px 4px; - font-family: Roboto; + font-family: Roboto; - background-color: white; - box-shadow: 0 2px 4px 0 rgba(0, 0, 0, 0.2); + background-color: white; + box-shadow: 0 2px 4px 0 rgba(0, 0, 0, 0.2); - & > .panel-heading { - height: auto; - padding: 22px 20px; + & > header { + height: auto; + padding: 22px 20px; - background-color: initial; - border-bottom: 1px solid $ignite-brand-primary; + background-color: initial; + font-size: 16px; + line-height: 36px; - &:hover { - text-decoration: none; + &:hover { + text-decoration: none; + } } - & > .panel-title { - font-size: 16px; - line-height: 36px; + & > hr { + margin: 0; + border-top: 1px solid #ddd; + } + + & > section { + padding: 0 20px; + } + + & > .panel-heading { + height: auto; + padding: 22px 20px; - & > .panel-selected { - font-size: 14px; - font-style: italic; + background-color: initial; + border-bottom: 1px solid $ignite-brand-primary; + + &:hover { + text-decoration: none; + } + + & > .panel-title { + font-size: 16px; line-height: 36px; - cursor: default; + + & > .panel-selected { + font-size: 14px; + font-style: italic; + line-height: 36px; + cursor: default; + } + } + } + } + + &--collapse { + & > header { + cursor: pointer; + } + + [ignite-icon='expand'], + [ignite-icon='collapse'] { + width: 13px; + height: 13px; + + margin-right: 9px; + } + + [ignite-icon='expand'] { + display: none; + } + + [ignite-icon='collapse'] { + display: inline-block; + } + + &.in { + [ignite-icon='expand'] { + display: inline-block; + } + + [ignite-icon='collapse'] { + display: none; } } } diff --git a/modules/web-console/frontend/app/primitives/tooltip/index.scss b/modules/web-console/frontend/app/primitives/tooltip/index.scss index 174d624dad133..a6cd79ff327d3 100644 --- a/modules/web-console/frontend/app/primitives/tooltip/index.scss +++ b/modules/web-console/frontend/app/primitives/tooltip/index.scss @@ -22,4 +22,21 @@ padding: 12px; background-color: #FFF; } +} + +.tooltip--error { + &.top { + margin-top: 8px; + margin-left: -1px; + } + + &.top .tooltip-arrow { + border-top-color: #f34718; + } + + .tooltip-inner { + background: #f34718; + border-color: #f34718; + color: white; + } } \ No newline at end of file diff --git a/modules/web-console/frontend/app/primitives/ui-grid/index.scss b/modules/web-console/frontend/app/primitives/ui-grid/index.scss index a83cb579fecb8..331b075be8851 100644 --- a/modules/web-console/frontend/app/primitives/ui-grid/index.scss +++ b/modules/web-console/frontend/app/primitives/ui-grid/index.scss @@ -105,8 +105,8 @@ } .ui-grid-header-cell-filter { - background-image: - linear-gradient(to right, $ignite-brand-success, transparent), + background-image: + linear-gradient(to right, $ignite-brand-success, transparent), linear-gradient(to right, $ignite-brand-success 70%, transparent 0%); background-position: left bottom; background-repeat: repeat-x; @@ -114,7 +114,7 @@ &:hover { background-image: none; - // linear-gradient(to right, change-color($ignite-brand-success, $lightness: 26%), transparent), + // linear-gradient(to right, change-color($ignite-brand-success, $lightness: 26%), transparent), // linear-gradient(to right, change-color($ignite-brand-success, $lightness: 26%) 70%, transparent 0%); } @@ -126,14 +126,14 @@ height: 20px; margin-top: -20px; - font-size: 0; + font-size: 0; } &.active { color: $ignite-brand-primary; - background-image: - linear-gradient(to right, $ignite-brand-primary, transparent), + background-image: + linear-gradient(to right, $ignite-brand-primary, transparent), linear-gradient(to right, $ignite-brand-primary 70%, transparent 0%); } } @@ -182,7 +182,7 @@ width: auto; .ui-grid-render-container-left { - .ui-grid-viewport, + .ui-grid-viewport, .ui-grid-header-viewport { width: auto; @@ -216,10 +216,10 @@ top: 0; right: 9px; z-index: 1000; - + width: 5px; height: 100%; - + opacity: .2; box-shadow: 2px 0 3px #000; border-right: 1px solid #000; @@ -299,7 +299,7 @@ position: absolute; top: 4px; left: 4px; - + width: 4px; height: 8px; @@ -353,7 +353,7 @@ position: absolute; top: 5px; left: 2px; - + width: 7px; margin: 0; @@ -367,7 +367,7 @@ position: absolute; top: 2px; left: 5px; - + height: 7px; margin: 0; @@ -435,7 +435,7 @@ width: 4px; height: 46px; - + background: #0067b9; box-shadow: 0 -1px 0 0 rgba(0, 0, 0, .3), 0 -1px 0 0 rgba(0, 103, 185, 1); } @@ -481,7 +481,7 @@ .ui-grid-icon-cancel { &:before { content: ''; - + display: block; width: 12px; height: 12px; @@ -525,9 +525,9 @@ position: relative; padding: 16px 51px; - + border-radius: 0 0 4px 4px; - + font-style: italic; line-height: 16px; } diff --git a/modules/web-console/frontend/app/modules/states/password.state.js b/modules/web-console/frontend/app/services/DefaultState.js similarity index 53% rename from modules/web-console/frontend/app/modules/states/password.state.js rename to modules/web-console/frontend/app/services/DefaultState.js index 60d82b460912b..db7b3e3439219 100644 --- a/modules/web-console/frontend/app/modules/states/password.state.js +++ b/modules/web-console/frontend/app/services/DefaultState.js @@ -15,36 +15,22 @@ * limitations under the License. */ -import angular from 'angular'; +function DefaultState($stateProvider) { + const stateName = 'default-state'; -import templateUrl from 'views/reset.tpl.pug'; + $stateProvider.state(stateName, {}); -angular -.module('ignite-console.states.password', [ - 'ui.router' -]) -.config(['$stateProvider', function($stateProvider) { - // set up the states - $stateProvider - .state('password', { - url: '/password', - abstract: true, - template: '' - }) - .state('password.reset', { - url: '/reset?{token}', - templateUrl, - unsaved: true, - tfMetaTags: { - title: 'Reset password' + return { + setRedirectTo(fn) { + const state = $stateProvider.stateRegistry.get(stateName); + state.redirectTo = fn(state.redirectTo); + }, + $get() { + return this; } - }) - .state('password.send', { - url: '/send', - templateUrl, - unsaved: true, - tfMetaTags: { - title: 'Password Send' - } - }); -}]); + }; +} + +DefaultState.$inject = ['$stateProvider']; + +export default DefaultState; diff --git a/modules/web-console/frontend/app/services/index.js b/modules/web-console/frontend/app/services/index.js index 49e7632e42480..55f8d3de90211 100644 --- a/modules/web-console/frontend/app/services/index.js +++ b/modules/web-console/frontend/app/services/index.js @@ -17,7 +17,9 @@ import angular from 'angular'; import IgniteVersion from './Version.service'; +import {default as DefaultState} from './DefaultState'; export default angular .module('ignite-console.services', []) + .provider('DefaultState', DefaultState) .service('IgniteVersion', IgniteVersion); diff --git a/modules/web-console/frontend/app/vendor.js b/modules/web-console/frontend/app/vendor.js index ea0f63c229413..eac47d4d350ab 100644 --- a/modules/web-console/frontend/app/vendor.js +++ b/modules/web-console/frontend/app/vendor.js @@ -25,6 +25,7 @@ import 'angular-strap/dist/angular-strap.tpl'; import 'angular-socket-io'; import 'angular-retina'; +import 'angular-messages'; import '@uirouter/angularjs'; import '@uirouter/angularjs/lib/legacy/stateEvents'; @@ -36,8 +37,6 @@ import 'angular-drag-and-drop-lists'; import 'angular-nvd3'; import 'angular-tree-control'; import 'angular-gridster'; -import 'bootstrap-sass/assets/javascripts/bootstrap/transition'; -import 'bootstrap-sass/assets/javascripts/bootstrap/carousel'; import 'brace'; import 'brace/mode/xml'; import 'brace/mode/sql'; @@ -52,6 +51,7 @@ import 'file-saver'; import 'jszip'; import 'nvd3'; import 'lodash'; +import 'angular-ui-carousel'; import 'angular-gridster/dist/angular-gridster.min.css'; import 'angular-tree-control/css/tree-control-attribute.css'; @@ -59,3 +59,4 @@ import 'angular-tree-control/css/tree-control.css'; import 'angular-ui-grid/ui-grid.css'; import 'angular-motion/dist/angular-motion.css'; import 'nvd3/build/nv.d3.css'; +import 'angular-ui-carousel/dist/ui-carousel.css'; diff --git a/modules/web-console/frontend/package.json b/modules/web-console/frontend/package.json index c30f80c6aeb80..100e644d8ddc4 100644 --- a/modules/web-console/frontend/package.json +++ b/modules/web-console/frontend/package.json @@ -42,6 +42,7 @@ "angular-cookies": "1.6.6", "angular-drag-and-drop-lists": "1.4.0", "angular-gridster": "0.13.14", + "angular-messages": "1.6.9", "angular-motion": "0.4.4", "angular-nvd3": "1.0.9", "angular-retina": "0.4.0", @@ -51,6 +52,7 @@ "angular-strap": "2.3.12", "angular-translate": "2.16.0", "angular-tree-control": "0.2.28", + "angular-ui-carousel": "0.1.10", "angular-ui-grid": "4.0.11", "babel-core": "6.25.0", "babel-eslint": "7.2.3", @@ -89,7 +91,6 @@ "progress-bar-webpack-plugin": "1.10.0", "pug-html-loader": "1.1.0", "pug-loader": "2.3.0", - "raleway-webfont": "3.0.1", "resolve-url-loader": "2.1.0", "roboto-font": "0.1.0", "rxjs": "5.4.2", diff --git a/modules/web-console/frontend/public/images/icons/collapse.svg b/modules/web-console/frontend/public/images/icons/collapse.svg new file mode 100644 index 0000000000000..86861a570ea14 --- /dev/null +++ b/modules/web-console/frontend/public/images/icons/collapse.svg @@ -0,0 +1,3 @@ + + + diff --git a/modules/web-console/frontend/public/images/icons/exclamation.svg b/modules/web-console/frontend/public/images/icons/exclamation.svg new file mode 100644 index 0000000000000..95e46137d5210 --- /dev/null +++ b/modules/web-console/frontend/public/images/icons/exclamation.svg @@ -0,0 +1,3 @@ + + + diff --git a/modules/web-console/frontend/public/images/icons/expand.svg b/modules/web-console/frontend/public/images/icons/expand.svg new file mode 100644 index 0000000000000..569c9c0643641 --- /dev/null +++ b/modules/web-console/frontend/public/images/icons/expand.svg @@ -0,0 +1,3 @@ + + + diff --git a/modules/web-console/frontend/public/images/icons/index.js b/modules/web-console/frontend/public/images/icons/index.js index 816ef1f565376..1168942296e59 100644 --- a/modules/web-console/frontend/public/images/icons/index.js +++ b/modules/web-console/frontend/public/images/icons/index.js @@ -31,3 +31,7 @@ export checkmark from './checkmark.svg'; export alert from './alert.svg'; export attention from './attention.svg'; export connectedClusters from './connectedClusters.svg'; +export exclamation from './exclamation.svg'; +export collapse from './collapse.svg'; +export expand from './expand.svg'; + diff --git a/modules/web-console/frontend/public/images/main-screenshot.png b/modules/web-console/frontend/public/images/main-screenshot.png new file mode 100644 index 0000000000000000000000000000000000000000..5e1b9e9642de61b15e0ee2e0362d3e52188489bf GIT binary patch literal 86830 zcmeFZWl&sA*Df3&NN`AS4Fm}?XwU$`g9Hoi?hb=X5YvI@538UEkr+vK6z3Vjd5p$3~xVke6Qv5W zY(*NBEG3sJ6-#dwXl!p8L{hYIul}_~j4)m-I6g2utSFlk&F9E_Cjy&&`GwT?-A4%b zuCT>udY9);Te`U!V(8!NCu3@9w8Y=b^eN`%KVPMKsTKa|l=vSz0ow-eWO|vlWBv|V z^zwc?z+-=>oYBt|SGHpjQFAqal#b@lWlikALY?W4FqfJ69n-APPx>sTX9oG7KZ>;r zBI%VxW23x_S0SyKIxvz3M*K76#@)r2M$+z~sQePHeza{pRjE+Th=8_<@?=sDo+}x)_Mlr`|7Z-+H5HPqC&sz ztNlvzMk%OWWj5~Y_Iy9j)_tQJ*>=8CbV1=z&R?(z8}HAx!k=mz)@X|gIKlx?r7c+Z z)E!na(Se?pn#{zB>LjWTb&ZZ=JC*=R& zW43xWCIT9c>-hZhVsLOU-l@r;Y4y8IT)Zb|yxi_smd4J&{UCJfY`}NF{vS;UN>5>K zo>Fea8K_X^SC_5D_ovz`(~El#boYT(p~xsGB_@m10HkJ?6+dusA-iGc6SH>mxsf1gZD-# zoh51>qF?_~i^*t8SQYEH<5U`|EGrdmcUTiPd)OICOTa&>vzcRp?0spl7t(7?{N*fk zQ^brn#~|)kV?GkEa-y6krB*_y_;;!!b!s@ZI+~s-mwdmX{I0NKcq_-53T)AMwlIfc zBbQog>PaO3*dnd9hD(x(ipBIuep{%FVnvc`_C7HHbESH)$tQlQ+Pc|i+CVYUW9VBs z#)-3DQ|7XQuxzu_id!?#b<0O@zg3sM6f`8Q)$DgL&tcY%39A-(`_HUVegK3HG41Db zRDq|lMx?Q&!`bGTepM4)#D=O3dZ?N1H|-!bEAxuXn z`Ruwm*7LQ1Kyz1g&|f3Rv~gCO8oDH_P}gtvWR{(?Mrt1LcF5BLWx@xGj|FNuSJZQL zXjNl199-zSf$4iH%f9$=ZPhy$$)!1xJYdDpLEgkHfir7L?s5M}sJ!x~joJ_F4lJv- zsrdqGpx|9;Yf?A5OEvf;pvGJ6s$IWfzt-NSLQ_s=N}Uu?CZ_+9F$p@5k)9sUv(XIQ ze%La!^w#RrbHgqaA*Si6TP`HuasvPS#&*Sg#g?UUmF}!NI(6~npV{CA?E`1sGz5XY zpYAYZUpL_YFcWmd`;jKXFRy2r%p$|4br~a|CRp98rH;oty^#8QgSE4KHVC*@d2jYh=p*H->Gwe`j03?aJIL`|uuagwVU?HB8&u_69xN%MGR ztX9{cTiwf^1{VGbR!H$hoynR+?sEjHr+8NECbnEi(bowE zfLo9$Hkra(C*TpymbcK?vnV0Q_BFIERLQ)`9I=z_EJvpuG^mTvtuUapWFvY+~{si30(LM67`3m z%z&h`u`CgTZKnb7cC|or)N+>JH=gr!xObQnbd7_69oPm|=efAJW?nbHk!&QtVfiYI zklJwr&~9==JpHJ24$@F|9)8BUDI&JiQCvD3$6Ok+4hF-F@cD7dZ5MME))u=|^`uC@ zODXn6%#eBvX9iio+>o53jJ>xyD(N8{tt1z0jXI^J_`kKkf=SYwmb$6@eDJIMVa1Hn zflyW1flqt2p4Eynq&U+hKJ~4IR{4A z%MTQ#sX5@q;F6G`k}dZYwTF7k>d1N)8IY!sZpUJCRK<7gteo8nEOF>ORvkhTC~uQP zV+JJ?yd!JOJ-GN5b?PO-Eibz#E~}JCsQJPIql(4Gsf@1I5b7=!^z&I%G>yNx=?xe% z5Z-$y$@udck?2S9%ra_6Xt?x)L~(WZRU_jLfn@juQdc9*k278?cSqn#qZdnH5#EfE}$^vE?MY33DyQJ8=3E zCq}f;CBoo{koyapFPQ-?Wjn%m{H>I}FLq_8&H9CvX4ld#Nz8@b;6{(Eg(I%CTl6J4 z(CV-FJskm^XbP-gAeCzVK^&3dahRLUQS;VhKg?hbS{Zs)7=VVz@ZwfGR_kMp15LZF zpD}ZS;!_N}10~>F(-{YGXg8J}8{E(@ayDHCcLm)JlAYfYplU(&vvQ)giVg<>nJ8Xs z(L3~DOd-31-mcS;9Lv`XvJz-noQQLrx19&!wCrom=p%CKV!xQbI5xc^kKEK~mp;>S zGS1Mf^*G>`rDyBhK675^vGiR>lkX3kklxKnAm*j=YtU6%c*IxY0}$tD6y|Nu(puiz z`^O`9d`Gzz5I7~|hJ+j9pX15HfXe22D=$8ko?+)Lw6e0V5ui?WQ2nTLa>#ziKf09v z#6SHGX>S&f0;`qE{7SHsAjFYLIzyREL*=9^BB`=q%ZlaETCg3>MS7x;Z6p6;AB;X} z;URP`?V5!j%-cp`m@~>+Sa82})Ys8_$w0o=+tIM9`Cx3y#J%|DcNi&(v5XbfZ3;@k zoF4E7H6Vbn76iTE@qDnJb#u1Zr7l%yt0+pX+<3`hVW)+uIudB2#34U~@3~?(#Dq;S zmvAM`@N42gH_WCVCA+VMt*6ySFd67e3?-D#_B7|_KA{;lxT2O{#gNq51q=$<72f2M zxMgpJ);}UmT|&oo2`}qy!_1ABRb_4x^lol_DzvLlfVedmrhkOLK3KK%o~M##)>H2R)Hq zUMfv-hl4LU_|OtnqRGf&9-(}VAiB*TK-M=19DE zp5>%g+-!ID$!aif+Gn!{ta)D~XX)^sKyC*fqqU%I^++UI`I9zy`ey7Mt{Xcg5gO8o z)eq&*nQG%Jr2TVE2lE&Y3dvfXprOrs^;Ti^Gbo#BwQQJUSneS$_Q_}Y&Dl{UOVJ5} ziGG&|t+MzhgT(Hbu2RuJ4wTv z_Hl%Q7amKelNKc72S;ZXS7+F%c4A8}sE$F90K%~T3E{eKc8J?7VO6Kl)f%mE_AaKI za{+%!l(?g0L-T6;@Gg?i)rvboaQ<<%O46MA2Urz2(n4I_2==)x^l^)B+2#1dTcTsS z2PLBCzhT9%aX744C|;Z9u#4X7i^lAJ(~(qx>P%9k)7zKbmt<4QO2H;n06lBG`DrBk z2^9G?$L(xq1gvRReOB|MQ;FaP?DuN?s5Me;3Lfl|KK4_;g{Yz}yX7l0wl^39?os1qQ)MDjAr~-^UIT@Zb%~5X7y7l;m zaR&A|`+Q6!0k)(zw5oM?!XFV!8wt9o4k%@4-Yyfpi$Pbrk$ZTzNo0U%Pf7e8Wi7E< zyn#7s5E$$R(C7e#M(qKXe2!q)#Xu_(C7BVJOmxQA-GI%yoTXk&3cHn)I}Z(ND9+Yv zo8OR9*<|83vn7wU&h&T^K*k;Fva#*-R#zs%w@q-$4>R& znI4LocmrZw1Dg%YOLezy^%xKcB;XeCcJ&gvlmuywq}#ms4zWxgX{5?O%j^#-XHa&1 zp!v#qwXs>dt8Bazp7@JWRh^5BlZ4(4<_Mj+Qodf2wDf~f674?CK6hwmKbyRi(9hEL z922cMxXST#5dwtowxqUFCh2Rb0!$kYAaoC*W&L$m_>SBD{+oj@t48k!{+=dKE7kiy$kj5$Cuo>LbmktnP_M_B+VbsR)4YEf=TjqrptNrxI|=+fe1$wzI@=|R!Ns3FlPRCt0W=YZqs&FRU~ta2k0 zNB=rnfYVTrpWmqK3-X}XsRt?+scMA@??U48`2Fy{d96g6RaVYS8h}yfh6Qlt{UV70 zAY~PeTYGz@TRQ=CHb13aw447zco~vIa?gk!q9|EB;F>H6DgPLqx)dp%cH#2*C@IJC zI!7{&`c&_j7KBrvX>ueu z)&(gpzTVT&81~C=XjYwBB&gx>dvf)KEunttP*EtF*FCvwrOkWAnq#_>swVT}N!KZn z(^w331mz(%Xn<;6;)uFTYdtDzQRt8H(oU4gEA^*gY9Rtj^XA^K7O*ghy{1S)(9_Tc zq}<=gCD3hf77ZGfD7urH4OFqPwx{TLLlr z5HEY|cfYe&e}WGc61vvH1E>WZuWaO+OCfZ6O*0Fx=xQ6Tw*nv;jc*_`HM27l%&|G@ z>O!C}%+Vjlj6WeYa1z2#_gnsQ9Udj+DDDH1+PO{Y_2U!nnhAx+nPsj`1I?PZm>_Si zz~4S=V#Edx22=#ho^LE3EX!;QZ_>+&CoClm@N5tqy#~mQ%Mk+9XOp_xZoccZsHI>Q zr=+Bycq=EV&3#l+i9hycHk}7)Vzo1!dYF%ZT0# zuy(aWL4dbLe^9vWYcDH_@AXONX)Ji4_m}S~AVIpp4ti|coWlT}dU=vLD|rXWzNq;v zr!{ais%v3W=nw}+&W(@AASW9l^hmmSRk7Co6VLu1V!)pO&O22o2A;--M#uN- zJlo)Ov@qLWYew_f%L!pZ)?C<6$;i-a8XYs;=l6d3mr%04CGNclqxNH_+XxnLilZB- z7H9xO3>^zuPZ!DN=%`bGJ}f!^UZylO%{3gMb6g8Bk3JPAl5VpQtJ4!W#lxh5UqQPR zKV%p7JKBX7S()I1EP`eGY`P~M|2&r45k!y ztugMuI&;m}e9o|z{_A~ZY64JkIRB$LJ+1oh=d9dr&#nHm`Ecj?pUs<7v;S;ffYtt^ zsjjN}Z?g2&$c*$q(r_aF`=);j_TNhT|2lu`>_4XOe^Q7wTQB%z9fl|hG#A|{WDyM^`xk-i!lYF zL6q1c0lyi-(%~$7017`tb#~-dJ@Ltopa*arT6UZfk^%CjJs0fsytMEbi%S^$>p4=Z z3&?uMY`bjQk2a49xX#nBu~$>KBCzSf)v<`CuUMoEul?|T+HqV4%p^bHZ~=3wwvC2# z9a__#o~Mw7xCg%`m@NJh>+9`k*N+`f4x==)aPE!32R>69>U}yQpwqLJ6Eb1Er<9CI z{2f63dn$65^26%!h&Fh1gB?3>Q`~Wdwb%mX=m{&Sns7Hh+V)8h@{}zj;z5h>nC@>F zbTc?fG>|Md7iXWwn`N1!fiRpZw)o?A6*9#=0o(k4sQ=uhLMxP$wabmtSNxMye!nu5 z)wCM!*Et*$X+`p1itA{!rMEuXzuP;IIVR`BMe1AUKOqSGK~FpO4tsd%GPODi#$eT% zz;Y0Q+4VhU5sDI(9{BBN#!r=!pOBxij!n8cqyP&2t#Up8b=lB1%lpRa)3-v%9S6OG zp3Nq%Ay*hLHJg|*Qd=`TK8&WdNP^;cS6r}rZI&>B_v(GlD;%xu&pkKm?4DhPT(NTq zIU!X~tG%G}e81F+`7qXKGe@-(`T9)Vc$W=D!JhS}$}#iv{+z8n{~3PQ(yb=d(PF*^ zpz})%t2Mi>V8vYAivjwBI}QxwJE6eeet)k3&Uk0KzkPNZY~biDI@S`e6%0H?Njki& zJX*ycFFJdL$A}{>|K-XSkLHm%*Ic71svPZlfX1)`F^dQY9c7+MUx`j&Q^5c1D>Axb zhNE+HN7H){jnx;Q#6vC4)>;dFb!8KcU0ye34zBgRT2xb<3;lHPD8|S`{>^T6WPAg` z#?o&g8q48>==)xUSEZ1dGy@!3*#p?hNVU5=$4Wl!qIJ>}4bI9#Gpg$l2Qlr>L6<uP5FFt7nmm!lr*&UL^|Ty?(x{nz^K%5HUohpD6zs z_THam^)h_1q1$2pu|njnP+wgoYry9}xn5CJswu9ByEs+sF72b8XzFgEP3*)LfWA&G~FZ#S- z!d1lsP=E;tuSJcSUwuvqb%6EcRX%>Hb}qME)lq@rKFku*N$B`(mW~z)e%K;o#7vE3 zu5z{}W#O?CZ84jQH>=}uq+F!nDeK` za;mrUAsf_DGFR+AE%UXZzxxs0H-Exa#ZHf~lvkM7qwasrwh&Do{3q1qYX8P?F8hSq zJ~pvc0Nr3st_Uxemr|}XrStYi(aJ8n*&W0)K14yR47D{w`v)P%o_U5KcMQ&O)PCdc z_t5LnU)xMBYqhOy21ao#35Rb9T@XV$wdYV)dok++4En_Uy@67`-~bJxj<3p(mBD>_ zgq2Eir%uKgx$V!ZIxcXZj!_`%toU>-auK_re2rlith#7fTzRIap2p(_JfEZ$%nnn0 zSEcgUfswkjh7$*CW*oA%;AUARM~%CbWR!UU|E3VOY_M0QpF)U^g32sS*F;RUXQ=2lg zy0Bz)$XT(h6{$rj-)dA@EKfA168z#OinOIGF>aN&8` z-%Co1Xpw6t@xpC3=Lgd7fHz@465xUCsYSBLv;xqZC|Ai&v=6i%VdKBREpd>-uAW|d za{HFr+piuBF{3{%qpf4i$YZ;>BD*MJdCP8nNlnSCek#4=ykWb*?ysGiiZ%VX1yUkBR z8tlX#ImC^aMUc#eIKXxL#IRfIG7nz?bAVQ4$|LTXzpU7)*2%mTVeKq(FSY3OI7@r17()~oQ{PfZ()6%IS zd2~6t)cd*e_k$V?`QV`hOk!g6@Ch75Ns*kr5lf)C(Df$k3yRy18SGm$vhA8-Ot*O4 zKaZOTkAAt2I)a_@e1mNDIac(1Dn{u|dURfD-sB}O)G?69D0WiJD2lP>woF_PF$8xY)&?Y+6*B%Zu0{D+V~<%Kl(WK84?q{^tmr> zB(7J?Xn&K+Y|g9S5JgWiMBn};p@q}%@~3OrwIkvI)-37EKN5lAv1~Ob1S7s;P$l-8 zx*nyV3%ZPKv0V(y^hejqy16L&`{0i#-d|>^?;GoV>lct_v8Y(Ez@I7<7OTpp8d?M# zQC%SwLY^Vfg*K%ZYq{nb4mnaiAd9;Eg$o&X3UY&J4ER=>J_TU(-@dtht-`beJ~#v$ z4mo%kwNfLc5e6CW5>zdkxUj3cX#&&came%9<~$}jhjp@o5mzA06Vx_yEok?>j`=>) zNy@U8v$pyntP6{tUokf8+Ac_WitNT9-3KwxlFyRoAqg31dbQ6;nsxH!Ha(mlTChQ_ zP5HA|srDSMMFMWm{d5AEA0H_5S(~$ik_so|JXgv?c0$pb?5G0wEfs5fi5NY&Naw0` zy1!>J47Ub%fBzuI%{-^Iq*VLurS9!gk1<74CUG#15}b{M9fpM6`(9jso(evZi(^<( z0iKtP*1^%{-m%t;`aJ_2PeELP$>*64P{ylYie0OWWh}{`HWdEJDN7>FB_$;<^y3<; z#?v)JI+OCg6`bq;!R)e~qx><*bjSsqi9jhVVzejxikJ2sz-dCS2VU4ht-V z*VWWglmID$l&|c%3)%>x@>5M34`oT!+xMceWYa7?S?c#JbNws4jc;Ka#HH#cpn;j! zagc5O&5<6rO#leMxFSeKWd5qiMTT?#9!zNuZfrcaPcVQ!ZAHD82TP42hF zDVz-i-*_qzx}SDPQ;l1<=f3r|(9gdGLaJmRY9(}s%3x2+u6V0O(7ivVC z)63v6T@=)W-GB#Q>T*_nI;*V<)#0QNr>sKun_4>*>Xl5w*=PvAsC37LZOBL9_glW) z^jKxv9j{u`eCpqd$OTh#qJnRd6fR)1?<5`vRu5;^m5SFsf>5-xn?-8$%I0&*aIF5%3#Spk+N6d z%T{tH`vezVEQk?Cm9YnkSZ2x>cizI6sH4d@6V_#kZ=|ZHD(OxMu7?F8gpd9*XLvJ` zaR9tW<{=pm%gpSiF^O1wxJP4GHdPsD)Xu``Y#6Of1&{XAEg3!@c%ve_5Zo*9KLfSBQuT{ zJ^xE@Oq;TY^2&P7pCmGfxo;OwO~-H9d1E1=R!F1@_*%U08T^cNvdE(!_vh{gxK$Qx z84WFerjcJEUd}p^wzTCum7P|zmqEqMNcyziMKNtwtYrVlP&6?#--1U{{`rGuA-aKKi`Pp+Av3H?4V^X+)^f@I_6e0(vW^s;$^+RlzzUvL#? zBMs|7#}>AjHsF{}6asbPNM7B?v?qV_hlN`~M>)}P^oY5-;MqhHDm1j&b;muVMT$55 z{BU+@4CS_P{iZg2ofhLNn&mei!{@h+IPXxCyB16kIop>%LzMZC*y|6g`_8Rs z4VM-%%`?1=dmSx+IpYHa_quL0bRT@V{N?&LS|s@V-ziKxV#xqZ*f%(wu)=)ZJxuvB z!I?qvn9x&eP+IG~CXq>j5CFJlk_B~XlagNaYl6C73(@GS*I)1@syyhEL3_3ahwFf@#Q~Cf z_u?VYP*1Tu*6q0?E{9@2qEOrY_?lG4_odx#KiieJPw%}Ek%`ci#s?_x#}0ap^hSaY zFv){G3`%`K0*0Tw{cU9?)FsE{#Cyc7>Vl)}+g>pXEw3V;c{}El)fVdYD9)_Hr<%Mo1vhdxGR=1LB=-~$7e9I=b z46xHQ*vQjU8^RP~0R6a34kBKX3+Ndt=sfFV{H~`=G)3{!;)wpBUm@_;qub>>ajQX3 za4Wa%a%ugEdYx)O-AlsgbLp(M0UAOg>WdC7>pIt*A2xgHk==sTgDr>sulQL=b9l;e zHHu{9{;B>2KM(L|WA3CQDd3u_+#W=GyKRMo@g|U9-+kJR2WdcL=bcLus~0w9_&QEE zM~)%IEX8=KTiM>AGL_`*$qqtV8e}X+;ir#&KnSJ};W(`CJ_XTOm?e3*vy* zg$wGFKMR#9TC}y@Qi5@*5pO^*V|`vme(9nY_Z97S$lDjXKgEC4yH_2!Kucn^L^%`N z&Pj70N}>63aT#B0xn}pQbZ%BRghJ#$L3cov`+;sNUmm44>a@sWoM??EppM22Jm!}n zHItb-iW(M}9X1s81);n}HOwe2wy~Q}+w}_5c>=3KgpxW~y zDcUD3k*S;e5c1$*y~SUq)AEdjGdU^r+ruw7F)8D%%zsH09#sXY+>oH;)hD$u)b8PQ@WQAs3|5cNJrR_(?%7uiB+AyDYa2?c z$-dtrp50jf z`LL%|yx3HZR!vd0{i(Ik12rNUMj4#tOBEN~)N_^vG32piDVuo%WYG=I7FQt*-YdzM zd!r}*rued%Lpa?i?o!Ll~V*1rcG-x=S!m| zGKv_Dv;UT_PoG!*ck&j=sDEH%HnD?sG^=x66`Qy+G&kFR`qv}eDby7hYaNddEvG-( zALe!{xj$0m2`b;R2>xADMf+CxqPoHME91>T8zA71XNH%vOqni*4&nMi7vBKC{zY^} zPS(F~Jf56Xc|k-}>hiZYl(bM#I22OyeTaG#|DWAA>w{F)8CQ}Y|EQU?hMu5z`_H~M zJPA*Jlk%_g1g;E*|2JYhrd@b#>fm2}UEImRi<4;H&5GUQwZ_XVtN;N{$>@BcXVi*V7_ z&k1)&?C5K$k+E(5B`!}}P$$Yh;0;bHlyC)7=1tJJVh?-Qn!G`7F8^m~kP-%XAT_5F zN0ipIzaxQ{{?4bcr)?o~rMX$=Xq@{($Ko1EJMib!>9Y@<<+#&`KhM^nWSfb%Jm zory78=gDSh@k>|KW)M(wyu77%QHZG4RzGd{U zOoRZ1_?Ve6j{#^S$XRh1-onqMF0gd&RAjfu9hw&~; zpo3JSBRSG0jHqw^-m={TW06uY>ass7=i=1Kl3rt;ETYta%meTztk5*G!pC2zqka5Q zXY#Hly8)}dD^YDW+>{GilI;5VCL*ctr?BqQ`A4k$O%c(vfvd%dDwlEd>-)u$BYM^e z>vj3s<8c7f5>PWShU}I93#F18#YaSq8PCqE#v)ZGw#NWbCn-e0D0z;$KT$wZ*@K^o zPUM5X0ky$!%flk}(_gZ6OCn$IQew9KY`^VFY4NxT0T`rdI`!5VL71~ z;e&7BTX*^xmYrEnVfRqN+2mA$Z{4%VESu%2GB1e9i2fTXMX#T%)YEaAr=sLzZ? z+FC5KrV|S=@`tjdA2c1z8k;#K1-W3|7uy0O=MHB(e59;zO~W3o;z!B9$l(Zmy?b5b zfF)+G?I!R-5xJSH&a1x3)El3He{wWB`pDjIZbUuEQx(N_c6aL^IC+KP9(>D zG=|>&JwG#z_IojP^q8v3t#=CRmgd?uv`Wq;c*2x;K%fopFlL2ci?dlz(u6H@Q0#gr z0~fgKP%p8dJkSq2QuwaFjpk0U!UnNqXb-$xz2tISX<6~GU zNaSm`Uv#eF#g$TK_m+LKuxnQJbhd$Fo@yB*>Wx5r)I#87-%*~DyS?mJ$2a^4Nj8lJ zUQU~(gDyl-`krk>H;Fj>E6$3?f8vWo0}>pmTZbi5j)nD8G+qX6fEQ0M&6;Bkhm?aI zoZO&FwU_!^aY7T?85#-dilYJhj9Q$&`1vkU>yzS(Fk6GnDt#SvVfGj0Ql&;qpJplB zt+Q7`YXcJP#tc$h!l+T>^&I}r^=@P> zR_wm#iOofStA{w5;GDs1_kv;*s>edL8Dp7;|G{G8oe<#BMrhoeT-!s9A{)fDj~L#t zIqr3%%k4gkap@**bgIs>$U+vWeoK+1HVq`#_CMuJz5jq!$7abR#MDs+3Ux@_MDaHxc2aqEccjT4LS=yG z50})bHvrL6CmePEm-U{+gpW}H#YalqJu-GMXHOBQCTNF&6xq(-s9qy`=@ zH@Y?L&>VQqxN435Ign``3$M5N^mQs%na>Vy&=ld9cqV%aCk3mw`)8}A#6dRdd`Y65 z8#58^EWfX@*GGNnZkED|fyY%>J*tPF_IM1#{?x50_ZZ_k80IG_Y@XgWD7D^I3iN+V z?CjiUPvgbfnOUib_=)y!sL~h{Pk0}eTierkxx&Uc1m8{YUya3CMQckA zr4}Xrb+~o968>Zn`QM118{9X#_z%2|_}|F}z{*ZWFVWcuavLQwL@)9*z4b9bB~!l_ zf;&5W$lNq105XJ7zORwPgx!3Y@2OwLYx5oBs}7$H$ZmKtH`H2H#2H?ivpaJ&r3zG_ zy%LXa1oYOONaZ#%{T5@;1eYOKiuEc)LIRMCu;#>h{MBevby)93ct8hHiOn)fkW45V znz)a)@c1mcdd5at8&QlAD>86VV^9o5V8p>403Oo@O|0N`K z@BxbZTS6}3w`lX8wA|xZiC|lJoVC`~q_Kb1XE9mj{qY42;c#cfla)$byevAJZPoT> zLcIMgzvsd1cs}C;NTSM+Dkc`RDSgcDT!INrDq#fdg1+}_mAF+RFG_WY6^(N0pHAn1 zh79NfdDT-(cVoQiiJF2)feQCF!*j-TsbbKZ>l$J#cPY3s_uv7Ax3=cBCedd2F47mg z*J4*)wYupV0%VixcPXmmFE-xVj;cnS8GP-HD_ed4{pVWb&$@l%UIgK@yY*>xL6`s; z+tZOf47kiramNH_7T(H-^~W>MzR zF$oo87UK>DQTwXbWhltht|b^ON1-b9y06+fkd%vTX5@e^m^gv9=i0_^YN+`L0j4*H%&TGT%74zV34#OhTRt>S{u!2mB@L+MnH0-sBcs1 zh5U?e7qxDG%jhzHe%jefwEV3_-quC;5x5sMgy6?4wlN`-;;9GjH>q@N2@P%^!V{2( z4NR$#C7qeOD@gRp-n`v8#@0d3f+eWMGtH(f<>8)4#-5T+4B|&^I!p!bE*ola7?LX4 z&77B93az5I6n>qX{BU&CyCCrtCjC-ReaUYHp*X8T{Gg%uX9v?7v-G*Q8zMa>)7n<{ zq>ILdK5+>;z#sSCk*-~4NTVbsZ4H7Jy?eJl)w0u}&FjlbT9&GRb)}{p8$Hd+p_%k8 zP~H<7ng_RPWc^O+}7Mcn~>4?1*{SvWz9heSS;+SCG4D$^#U0KP=8uNzwMUB9=3 z8M~=+=jPETVvsbg0VEBTUbz)!gji`XGzuM}ot38WUGIxsvxQ36cP$y1(fU=1bhj%= zS9yAC0tq*ERCrX{F#|g8$V+W<;DjXR*(V(wJ9LhtjRJXZOh{C!U%k=;Jd_$BSO%kF)DeSv;dY>D>~Ni{2>F=Z=8?EI1Zx09rMs~E7L%$q4eFL@7YOt(aVZ1 zQCViMAE47j2nq68BfK`*C`5H?{w_pLb@gs7b90g36Bp8Z2La_IVCil_j_mR>>>*Eh zk>++r(xX*R7M`iE>YPZWYImy$b)BYO8dPGV`ZG(xGJQ9c4{H)bbyvHi;k#zCztJfl ztE`We3~IBd(Zi`q&+m1&qS&*ck0ay5@{> zOdmnl)NkoYhiDVSjRyYe|r+@pD%b5XDSBbvQ@eP?GKS99=O(`f7he^Kn7Sc9|GGI%@G|KMR>w zw%({d>yzV2t0)@^wdo8~pz!OcJ6)RY+_-rBku>Eyah62;F(QXhB>F&5KhrijK-$$5 zEt6e@Z8e%HT~^fmh+0A#G?{q?ccTPb1yA=68rfo0SA=?)TZ#7e^L^PeUl=I~N;gr` z5Qe?ew8d^{+H7UR?5u6t2#rccVOXdT)P6cidfKj#0qXNo@!=-q=%2Z=n{f^tJ{Rxw zak4}84lR{w4OTR%+bu4qZQX)U6+FBcZ=z~t)=%U06t3=TW%!~I29eSI2$`9@D3uAl zSJj`|6f$5E<~>7NXArSr7LT{7xc0y_j-LG$^a0Kn^}>N~L#%DDe{87FNs#Mi>1Mc3 z(*bj7c}YVX$#c5)TMa{FY=$)@`N|}NP#Un3vt0ERg+#XJHf64Ui2Bh(F}9`; zZZ@B3y!~5M%;sx5sOSausgkgke|U$%lbq=AY@VM5K2S2e)b<7XUzPl=^m`o&E_@M} zpYMj0SRVXdX%P98szz)Hg~^s9q*YCAoD% z`}Tjf_bm!8z&cbbP|s7JWUYGbw&ju0dkq<8ME3U7$!|6c+_+R1y5-0vIa45|UY&|| zdf<3qRt0#nvW=w)QK_#YB#)PhJwlRBXQ;k4XNNyTkL*Mpr*I%p_ADD(j2>>nBVP$T zry2Vi?H9)8k4pXnRn_`2sfCr&Ln~3mP9oHshOj+`G1n8BhW#ho;LfIv_kp@NoqMoW z)4AjoT$NO7kscmhqWh0IW zsm8j3|J#kP_Tlf=C7{vRzX|{nTbc51#u8X8L@6RerNb|LsW=?TnB^=Q&%e>W=8GvF zo-je@x?eA04{AzPb3Uts?l>q2t3+Fg?B7%Jj{lhNEjw-1d5PUT`7rs>Nh!FX-X6?9 zJ~fEF0^R;Og0lAfy$}r*bp5jEvljyWOq;-z>%*X=bu5pnz=eCU zD;XP^H^Vl^)%iY?U#Bg3nOBpgT-KxH0nI!B%ho36Z z4zo2RgM7moM7|K2kW)3E5mSENPn}X)_q^2$9Q3=shABj}RiA*yYEJZou7Ir-MXln8 z<(uInFFtqjebkVMsf2*W=^iBE&2XoUOQiSZc4RG4_vun5g0?Ipa8n^b}%2gk}!mq#7FL{a@kXMr&XIN%zn$>3rATB z!52CkZ^Dhzv>9IWQZ>Y1k)=iwiS^vAPs-fc-E-o1eQqbdD!c3A#&3bR7DnC*Q}G;Y zH=_AfsLL9l#yC9fc28f$0ohW)52~oL!jr0mu;-^O5pi7PI##=?#c?0r%qW+ypC@Z- zr++yYnbw*a?A_P>$k%cy3HDnMR#%i^!nSP))NvrQWb)-0l=3kpch>Qww%p*x+sar! zZzM8r6QWRfFFj1uTyog={+0%1zYjZ^UF4~7K(AMT4{ogo&oWbDFL`v&Bs6GG&;-w1 zXn{zf4c2Yqg;pTKbX1dOnJsOQTQc(^IUEk-mE5RjineJqUWan4g6&O{;1E@Z27d@{ zA^OE7eiU2bp9&+0nO5w{tE1I@MvVEi8<_SR&a}BZ){tt{%ldDRm7ZRES`w`lW^`8B~LkrO@ zNfF0am*V%gdT-~B83rF64Ht+M&k2_FL0~_`;*ee!J|P6&1-Fm=%P4NQQ)1v8hu)-a zfTcldRD9lR@HgxaI?Q;83e;zWOFokZ;z<>YCENS^Dd1>P6kUN(I=}Ygi*RL_PNf)8 z{#Rf<%QP1?)k23=zctvS)GzE8n4YRspd#2}LF;}&arKLwTccXv|6=Veyy9rvK0zFU zH%@Q}Awclp7CgaSg1fuBL*tM@AZU=_?(PyCf;%+cIHd80>EwC7cfOt7nX`M&)*nz^ zU3JTK|Ljt}@UU(8rE-&R!g}dQKeZsZ=G_!BsoGbiMQYOw0gda&VD-%o!uv;6S6FQrB!Q1M>ZB7!r$lWD+~9nsPP?vxpq`*>ojll(!!Zu15XX@S8 zvc4L;o-CR_D$i_1JR&E%V}rs!l;7gnwV&7jdbXf>rlYD zZmlSaoR`p{3lxe>gtm~fFxZK)po5mLc3I*Q3`=R=<_82Sq zs&LKDs4*la5?_9x&6!=G@X^;Qk| z%&nKJI}KLa`WsquZini?Rjv{sw6*s^GcBaxc?7^*ztE-9oTfYh6rv(r-}m9CVj5-D zd_ypqbT8!>?y3q*?)x~BrqP1ax=K zb(zg$netnyF>(f~*|6ar%Z3N=Cpd~4Y4{VJB|d;>oS2J9lA1{X0R!sK!RkcuD@I}K zR{YJQ&xoz!+%O_MTWfbMAH@-Zd8DREMnW$x-y9WSgQ&MYgV-X$nIL!i=Wz#MlQp#3 zM-RR>*%hVm9)9Vku9!5>eDUQBo;&57JX0*e{JNSu{+m z?Ft{w!)>smvAB)$?*xRWVYS&(^fJ}@t>WMhbf!Ecjo!%m`*+UtCYm*iD0Xl_+wvRx z?<3zJ-N)?fe{-YTj6R~VsvL?tqIrXvG@XQoksj^5Z9xs1dDB1YTeh%SSUJEn3-Na+ z;l`lMl2>WoAABHcd4t=j)>o~IWF&WEeXi`ornR4yk1QOZhr4LVve)JAUz*E9yqlGbehraQehqllU_t!}GtpZvU0s=OU zqF>q|WCdWYFK^|0gF`_Wr};KTt`+@9pGC>Zg@Rr|X)}B6%ruKjmpY2th1CQ)mkeu~ zrHW^D$W^$LAHZxfUe#2qQ$E1`ohVl*_oCsRpJS^ft1dV_-yX^l_th%IIu_{3LKi zo?kDc9KCrk2>xD_Wi%s87tc++kD;*jE&$M5)LP8UjVs5Cgyh{L7B<;i2O@{Qo=9o^ zzUMYj4FppT3eZ z#)+bm9ByaA;mPI8-2kU{mK<)_mM17tmzS5v>HZk6uBjOT)G0HaYk8C$WNNiE;)ukb1jl^itA@%I ztWp*DS-&Hc`FcBuDQW(b=@>McUQ5AEv_D6Quu}J_#5q$zvO18Uzo^+dfUve`U)JV2xf4ja^J%!JHcE<#j9V`}o3NHahi2zMzI?qY{zRca*Wp2&4kXW25 z>EUc?)ZX;uh0jb)3XyWuhBlbj@EJppLe{$gz4SBL0ovp{;im&7KwZZ9nDsc&(-nuW z_5dx8jx1ihR441yal$WH(ghT&mZbxi9n7Qt;ADrFj~+#+kdn4Z7GxeRW_oXjS6C+ieN^Ro0=fs!Sm8cyv#YdJsG&OZh?AJ<%vM% zpDE0nD8A&ZY(;yiyVPU5_g4NuUqWYUDHQ1EO7D?_e4L$`ka&TOBIrbGV+Gsp^KX@r z2E(b4KI!z1{E{D=hM{$SKPzkGeQ&K-tTNt|ttZuXWd?U+wXoaUhF@PMXnmxa$YD`8 z$ob)m*2~;z{u;Kx-(#5KI=&AdGOal`bpF}S)M>TyvvE$u$+sD!d~8lN7bI?$BikF? z?A1>-PGqR9u*D}rP?uf5O|02!Pts5Jb4mS`L)_3Q-63BElDb-&Cm5Sp$bd-4XOCHwt@|=SrrBFDOpA&vZDXqn21x*mMgCxqa!Y(9#K9@LoGG7a z`ud44DV3-r?pW5UuNFhPT=nF(XR%?ikeB^?|DRIgaJ4vdh?9H5&Z zFkD8ufCe2W-hbd~n4)5r{U1v+_uKXV($N0@C~EmXiVsx%UQ?iA!=j&X7(a*>O@tAW z>*-G^VJfb@o%5D#TDZH~PZ@$GdNBVJE~(xHdy~sc1mzFZ2pi;Vk~IROv%ko}tu*Fh-)si%JbCRk#{*!PotIBhgS zipDiq9L9o&E!lpB#N_rPkxs+2zp(v>P>j8IXm5pW;#Y;`I2Z&7!@-x!Bf>9Zs&zU^ zcr7poV@XjBpTo6p>^-t>Bw*tic45Ip74QT_QZ#%56bF0ZcQHk|jtuT%?#s(PbnN6} zr(b_=P)1bSw}3;^3Fsbd0r$iMbPon-qX>Xo=QRo8e`Gk^@wim390(M&mJUZVy_sns~yqhHF+?h4Sie9&pbRbv>T!WuNnG^D0&G|#IB z`G{`~OG*~cfl?sXW@U1}^+qH?wcSS=`$eq4{?jjnBVde%quY4399OWoWPq)a;Xg0f zlig!pvkKjreT*2u=+D9Y8jiJMb}8Z!H+<8MEexA$c3g)QZzCLS!Mg`(+) zX_g^7NKk9d_|wG?aVz<8<4STln$KUA27W>Jocxwi4RGp*?q;xtX zF6c|+!Rr}3>FRJmdtFtYYLD7=BhetLooq(uph#^px!ALfOvTs4+!_Sbox!De8QA^GtIl9(+mq&l11{Q+=eMWo}FIxnsU}Q{IrJpXCL9Z(|{* z>?5B$9C2wH8;ETPKi6p`U%x5kVlFYA+=cN#n4coY;Ni7@*M|z!k$fB-G(K$N4G?-G zf)~7>N-lYGScc1aFbEx=S-7Y5ia|uk3z=%FUBhzr$V;RcCkg&eZI^jk zFs2GiQVG*3>wj`TBBbnB-9Rf4A&ow`0BLj1_uRyJX)An+m!g~1QvDpY*xdrG``uR> zPPv&iAD7R8bYaR+>%?=0;Ip`fSQhWuQ^3!B_BUr`rpK`fa%jiH83TXiTTzO=1*?OF zEKu_Dh-sVCv6sqEQ^feS!e~$2I@mpX>d^phRj?UJus71PNBjJl6Okc+?+wV?(5v}t zLMJwz1n#>93~U>*ro1`|yWfW8=AL{WL+aW6plCXYXD@Vz0ay{3`|(r!G+H8Fbzjt?x6;VX0I(Ryag3M1}wOAzhlo{}cI^3pOF zwordvcXidk%A%&#gOY&D(&VaPe0e;p zTrX+&pFOy^1jM`c&q0oR$hOQ_X8bfjjhDzybIBMgUuTsp8LHx2b)&o+Ic= zle+N-#a8BM2UV<&A5Ljx8QfD)I+!4uCRvc9RJz~DYGeziWYt+5k-KAh6s$p&qtjSZm9A_(b#Kn}1fR-9i?$>{w+DTXT-*u`x9Ci&t-~ zdI8x;X)3l!R1TYkU+#&&@J$zLqJbTD)wjA$V@Kqx#=Q|EoyxEkbRLgO6}Il9WP*HU zqz4n2zFVv0%XKDoU4*OZ+ia!NwiZHi6mQezRVPUql%5I=R=m`K+`^~rlEXxQk?MDU z6`1Lq0V@?+Yg-3Qq0V@!S93Z(4yzJ+m@;~$2ORn%(wCmJjg9V-o=fYkvV~3;UI<@5 zh>uy8&s~Dbwkh3TQ-X(YT;~FPRL;HeSX$IOT60<*Ee3r^$hSW zh1_+ffvRc<2xL-kuPdQ{f?^)h)i5FhdXl(pS&}#hcpnLC_CSY~@m3-W2{&+7Y` zVTpI$Va=Nsn(EznfLEE^{BYOg)T-%>nX#W#V>RuglX9-Ot}FDTfsQUtH`V-TBsR!g zHKi?Rw}Ku;o7(sM+VvbgjxwS$JD;`5nR&zx8?K!ww<1s_Kk(MngbpM`Ew*ms25{L& z&Li)iU&Y_lc+EIk@s3e(QP$yAy(4DKIqMsapCA#rh3}X+M0a+%(JXk+yMd(d<1sW4 z`eN~MJkWXW(jj>-t7q~fsAVIn-|;3P=xRfLss3y69+slgkmfW`R z*6)qzYBd%Y0tJRgF(q7)c`@=+bDUx_L%9c@h8zJOWhFuLYl}B09Dwgs-RFm5Xqq8d=ELzqI(fRG0Btv!+yvBg&AdJUZ$(BS+E2;O zkFp;jq(0$%-*Myga8O&|U!i23M4|bqE5vZ*K!Wv8PL4_+$iJVg_nRcrWVEK zOSsfa3r}P>GF{t+pD`)45Kh?z~K2DGC&wMb!pM6(lrq5+CVrmId z%vIRP`;_`C%EUlxena!eC0N~yIThj5#;{KpMDvdTx%i_=loukC5M@mGbF{>sM)&OC z3xjG5DFZ8*>HX_q{8LF{ov89)4@w#Bew7E*mzkhn z{abN@3npk=75f`MT`2HUWk@C9w@Xj^CWhu_ zd4cwHj_V>j8xEmnxDYPq+mG_Z_b^^TlKH+*U?-P`z;R7XD-MN)tu^oYx~C^L;lvcm zOWUOju(*-nh1g%MO^ap0P5^l;e+0M1@YVs^!9feyJ;oD0$IHHOZ6v6c?u!!{|@}kQD@ZP2D_nj5bTCO zgT}_>Zyz3*e=J_AC@$!um~2E32(xv?MRf>t%ZHiOVPBtT69U&a0wOwez&6MwKz~!A z2uAw789*mD#!Gj=#A_qPE~n#LCCoi99HJpE5<233nxkmFw8${62jhE7liziRBQ=$` zVzDvj8&wDC6G)?`d3ECq3y8`b6sZ7Zhii_{!^Tl_dp8V8vsMJ@Cr zIEl4t6RX`?w@fSFU*#k?4X=?BQ-PymlsaO2>uKFTdZ&-iI8Q&zbwJh2TMfd>G9;Xx zgBB+xOakGlwxVkfd60wPKb3Gr**n*K+>_j zb@#t-{Ng)So|qOtW}cduTwBCNSnoZFgAY7JsQVE@tqVqzPPm1n6GRO zpg6EMJiPnd?|aaJ7TI5`9q8;wWwW7R-l2u>@Q9}VIdX^FXJIhiZ09hf?TK?2LSMW< z0h@YjOn=5M;@7u2AyffVg=%+iijIztUle3JO9$?w69x-ng+2RnO|7WLrkc?>T)Vwm zY!8_Ni`4ZO2uaMW4YHk%4YkLral){Dx)r7cVLVQ5>JEb z%(P#fg_h_eL?epgUpyLXldnV)`47lWR>`!PR^>I+hNE3p#SJ#_z*}WyM)26St6Wc5 z=?ND+&}>>J+u=Pt9AMOwd*uAQ``9e7WuGj3gS-rc=(08A{9qgFPuZB8V-~1;X`28M z`?=p%ZF|C8Ekgb_#4Wy6`IZTL_~wD`a)$B1o@OJ`Db_GE6tK18WOQcn_O464{blvb z5}zXbA=C_`Yrl7m_8YP0N(Q~rL4+SPQ*D1~(rq@V|JG{C*>D%M=%kE1nk;_yV}0Vo zaKc?AK$UKiiY6{YfR`E*d$6k>LF#) zzvLwU^rmHmO7Bm>bp2-;!<-6t%8H(8SWwl~ZzsoCe9GanJHyMjexaSThxJUHX-p$G zCfH^&a#hLq+c^Z-B~W=zx{rP|Z^3@FgE^^k{ZjxInO%B<%$H0O89%8I?3sYgZo3A5 zt)vvz)LP%z4lV4Sn-_jPmS+v^$SAW~bdIqr4C;D6`#Au7}VWjn!E#)v?IS zf9kdCmQU-v_I%eC^-e221M9b&Sd8zdyq1r;_1F3+Gl8w@P@E+kn5f%-eMF!T#HYI` z97kjtt(;!Y0{xo)5R>TON%bk(pZ59P2&*XOmq?fwUVCHK{pBZVlV^iQGcU@=_j(eFS<{42mooodUf{*C|E>zrYInIn=+i%Ll?_Sg+^Y+PRZ85mJ~S z+ z7Sl{vP!}i=Rk@Y}r1OguH?^QmO-7%?Qh0SxEhG#juVNgx!D?*zM%zSs9d@B#D{o*4 z1g)c|K2fz(IrQ!55%{_Hj3{!~UARYkL;glXD+^MDV+EON!q%Iib1 zH$#S^`>mz>Qmv&syV+GlibXLeT(}}ngAaV1^2aJ2jpQ8??9>+;vA+$0bGl!WW|jdH z!WFwSus1Z#^2etP%h5PqTIg$nH5cHO2}_q1E+D2p5*_~8`dyKSeVj09dU@8>4-5b* z;gBnr_8C%PY6e!y*i%Lf44Tzr2U3o+orUh{Xc3g(8rWayhiCV|to>LgzazUN=fVja zByFGBj5@tZKk*Ug?qp3jKiVO|^!UUJ1@@@gF*nz$9S!-7iEWY-KghbAyt>-T9{EM9 zNwTwM!+8hDVmuGelrYbHY=6 z#II%K;no;HPv->ebI!>$wstPyKY(TVCPq0 zHqG{0BGKsX`P>+mF>K4tFat(=F=zE!KcmOU56A97-H)m^(V2wA$&lAwT1tlxg1G7PENt>f$tQ$OE>PJ7tpJ1I zv~*rSl?mu8!qIKHJXL~8w`XtvAJvA$8!YB)Kd&rR zot49`s`=G8CIT7pl!%xza8=ZdadNyxZnvSHxSm2d>3A2zlcVofe>ss;fZ~F)2AG6} zw>G%2nMD+(gWvLt`l6LBkcS*$WSbn_8^sL?=_LW$hy(*zg7#!7IXk1JGFu4_P=>ZF z7Mm=oT;9F3Qe-f7pN;#00&yZ9YR9q7$BDAQ|6FLe`@vA;RHQ}y!R_h$@0xf2%qEVwvnK*5#f#F8Qe`#M-pTj{gqkN z7fn^_^~gDGSmtIs)q?GG2LrZX^6pEsfvWE4@?4T>nJ3yAc*N|>!Ir=(qJ__mr)_&~ zOdve9vQwdoP_$=I?xr3kI%dVJF`TrpJGSjYe^fV1;ehMjBXzc@Lfd6YzZQqRGs4c+ z&u!?%zT0;tUgv>76Sx8>=TZYDfiD|6*2fJUuFbmLboSb>MzLphmt}ZLI((x2`xI&_sz@B@3Tn*xUjKD{Ixg1C*3uiwC4Ay6 zeUKf=W!xvkv` z@`wOL8I$WkJ3y-$IypqC(Umt-_b#?s6UEf!9XtIaA!cIMj~64?SN6`gkR(~(Q)76J zpByE%4Ez@YSz9cXbsaQx7h_#mDW(}ckX5u+ zh6w5BN*$IT3iE`Q>279j!x36Mtb!<=D=#iCmZ#{VOG!z|t*<9|8HbX}W*E`nzI`yw z?WxO-kUtZw=#cib_wIK^3D(9B`Bwo15yfbq@{{w1GY zh=@A}YzqO95{7#D|2Zr(ME&G-@9x)P=+2EcvDY!7)Z?(rNb4KsCBQ$9Nus0(mP9;Z zv8AWOmg}4dYw< zj}c|}FxPmuwF}R&+64?zRo3S7wR&d>)!cq{q|nUX79Du#v-y)$9QXnT(3j$VY{QOz zvr23bT0ZNY@*^NKjI|RRoD454v}wwA{krpA(R|Bxqn=q2G2(T+M)IJbk@&2%9VS?} zjptxjyJs&zjd~l-gX~~yZA)kAMOK0r${&6SmIs9n^XH88zgXg^_*aDG!UI@JxbOLx z^==GZF<}XQfbqk#M3#o_zH?{KvUNtU`ER#oH+dPoDRm^)=z-H%JA#qy%|Ad7toqi3 zVbwEBc4pDaHUU){gg?exKCHfV@d;la$Es=7`pUd7e$u_n*T~kN#$n2YwYyyFc^acL zno4(IEkn(M{Z}5g%6A)4E-^CRX#E%dhEa9KxQ=i<6e-!`Ob;~3B!|JJ%oV(uvkA;d z$GmvVO}mwO3vN_J=uI`8H6b&L)L}z*{r7}1xH~I2nnfVMo+zP;=xoO>KVXeAJp_T^ z4e#n}(pwg`i7AzE;d>?dd8i&ByPG z>pXG0d|+0`qfDl%{Prvi4!a%?=Q%ZcSOQ{`n?OX=O^4x6Z0Gn9!?F=?f9~M@cwlih zOa)OC&r^;6t>HHHc;v@;b;!^PS&ehPhxX|a9O}j{8ihr(pU{KWoklOVraP-hQ%GDW zA5e|)!(3ZJnmMn2wMUq~uDgZ4c|Q%i5eh91=^F@-r?{ckh1Vex=cKhIASrvsIc?q1 zi4L8QVpAlLOgzF(xyUw=C}voV$k@>InP=T9VFsXv*Jv!OPfHpi7>jrKrNAtaQ&;uXjk>EPxf|~*T&Py!@!D585S7(|HmZ;#+K1qD)Ulp!PWEp zquG$WYm7fI*w@AU);qBMLYtQGXr5BCGq}_|J%h!rYXQY$-^JNzMQlfN)$t=MC)YrJ z35A`2jU`|ji`3U7Y5gT|B`bDNRcwB&7lFtEEU6$nL9>0!uV-^Sr$Kh%oW5ob&$ZUE za+Z9xq@EUK+c8i4U?|tKL~eEeTkLGYVxel%Wx>W&&E>^@yyYgjsGy`&?X_pOMZwDW8v0lOg1Dhm*xm}-EovBzhZ4GKeDv0e zq-4aH)G?!)Hvi&x@zJyN2AALsJ)AVE1o7`r0XO|Lw0yWnoG|2W38Ni@l^Yorv3`DY zBhwWCO_t&JpONKsIyj0yH@tkQ*CpSbi{Rd=XuxS=Ko-?gYg4@M7Unqq3*+denLXB! zn^H~rhEc2^3(D~Qk78ud**MvD!<;O$J#+HPjTchj*Id>~iQY0OyX8?_*6uOEhgt}Ka!?t@y1#QInVsTqQ-dqa=%dTlk;z4ePr1>TlyC?F<1Ham-VWr^~^v#DDsdKM2@8*l8WG4N- zz~lGTUJAP+5-`<=o(v2><)?F6}~H!(y;wl1o6Z$tcxTHaP*% zRQH`uWmRX6PfkKzyjBdfx(Dy_f=QC*yH%BbU8aA$26*n-7>LAsQv6WcgtPM z5DhuR%{gx3D5^*ryCf2kSgFz|v>C^$?#qwy#jU%Zb7Q`D#&glQ^2ZvnYt!HW(Rdl3 zvVb(AMs%7`M;R?QNr>a{XZqVM)DvD67)s{&^))}dZChM-w*lfN^tZ?{UwKZyEAGv` zJURG+hv!TaBAkOSo`4?`CxxjR1ua2wULE<5{~g!Lruh%jbaWQo z53JitdsNv?$879C>AhcZ%Tz6z`Qo~{e1aP=_Dj)L(VSZ6>$~+IS@R`eKUrr%r5^YB z{l!M8+O6)O@dPSwNB}WW1?vYmCFdB3gKwsOE- z1>&q8C{os8!2D+MnX=HaW{{afqaq`APM7-K;=6le6|ig)qW`V_#*n=TK6kLWmi zxqm_^w+yKCuVMh=3b2p}{+4KX;`U2mM#20=eYzQUm2{DVwDbEgVMqYhJT3lNC}>=) z@_kOrqTlyNm^hQt<84~(;=EdYpexg32b0W~Si+fJXY1b#i}dqra(h9qSd0q^v8Wjww;&!i!dd54& zt4_0oaG6rOlt*ii730snRFI%)=noxTJgD++mj6PJMLp9{XD0{=5Su;p3ja&u(||`u zwKF!qH3-9*O-4Hn&Z6-Y4Kt8f;%tspqcrYmjWj-M91b6YFTs~=^wCG+5KO}Y&-|-> z>p7&(V`ZW;?XsO2+duO=ONzoue1pz92_zXBx9f^v>}dRbXDWb}$H5SD@?qyHtw+<( z`plmmHSo=kmWUnWjaHJhjsqMeUC3hgMX>F%ae?O8o#l)S~x(z^xm^e3}ua!NK?B-(K3+OY-4^NLy#t;k*{vMbp)_1(L_R1_I$` zW&rV=?=zNqp}Xc@p2ZEdHvBjIrf{mxvUk%OVW(%u945`&*i;Lr{LI^lXynhuYHem` z`|ZXX!~JN}@;PEGJH;%jTJYv~nVD>Gl?)`7Pit;b;! zMkfG-u=q5a9&FRHUXwCSbls{qH4X;lBEmV_9oDKh@GPuSJ7RWBinpEMOBj0$B1r7+ zORNPxi}3TM=R%p2TFsGFb1sLHpW7}Bl)$cNWnC=>H+j9>cJK>g5(6!=nzoi^TSHKO zlJAdLty0#8pe#pW$H(y+2*G5TST7EW4ZOiEBEJ^-*0EKEx@*&I*GI314#oo(wg}3~ z^J71>;S#VCamn!Fv^U;ripQb*W9-77qe>3o{Fr`&8DkNLlxcmO{DN1fqfr2-drGXk zGiPo=dCeIgFh_@S@qZ&6HGaodo*j!xqKx&lZU?P>xdKdeMeBYCB&qxcYdkGfz%XpRr$VvLb$*xvKc_3~oOqB)-hN4|=`hGzuI&k8%pt_A9$4Wxm(3R1UZbr!Juf=p zDO|=%ar}S5t>G@#kr~#D`RtpA%i$Wh^wAc1RG z*ZNUuOR0wj*lT=_w{D%}{5StGlm>&JQ81AMrkFiq{&0=fUvyt3hiyZ6aXGemaPz{W zB9Kf3$J*&b`$k`3xpe6=(Z>7$e!Sq9i|ufAa>e=xODf+Jgxu4di~C1ib?BcjLJa#N zX6t>ZF`8j?Mqw0*Ss~5lt{i=ynJ!e$<|O+u-Mq8_&v(QY!tJ>ACc@&`^`>YHHb}?E z$JHhS=(`wMFqXkRHB@3FvoCq>-Vl4>+?x7rr$`v4&L_3R@SSHf>dF3>*9pG8&sh+! zac`>iCE68Fn?z3`Qdr7jb+vo{l%+(q%kV3L@+cJt-?{$)_altBH0=3xu(n)|j}c#O zCkklqv>I4;+`Vr(_%RwMy-b5nW6EG|;*m4;o9->=&8Z9U@?2@Ty;QMiv>Q}|4Y-i? zRSklJzA!8EF|d94%KSYs>cuKZ@3~ZQp%_kQ7({Ns9@UFj>|a&PVe%`1lN-f5bNTN#aTuciXT)}?P-Q;aTN85(*cz=KJ~8w%T)@zE zj^nohF*Q^hrQFw}?Db*{JEI7d&I3N~jXhHYwo6CUGDC`cbwlJguY{R0yIPmfJv)@o zfw(yrZFXCH{irm|wCsnfm?qa{#1h0>m?xx>eH@XAPN@uf?xLUOAY2BiT~ZCo+kZYX zK5T)WpGWa$VClVwlvP_jy}SM6>vj~>UOPkL z;~0gaGo+S87uxuC4b0eC5Y1Nmmq=K>KBg%*a7C+R^Ah(S$v#apBa2ipG`C7M{4fY! z;sRu-a6EauzcGMK+({L{4!W|_r}7(3zRz<%(CKdcy*Eac9F(0mfx^tW4fVA^Kv*xPq z`*OuznG`5SvugX*9txRg3nrg3`!cCO)pQlVR17?sE;QtIGlSl?$bN(mNVb=GaKo3e z2s#mEKUXIQ1&{#)iyNjWGJUsURx_uYXJ|h7TQ%f2I6XXo-^mg%}#W9 znRbRxLn4#p_ZA14_%gA^9k9MqTwbVgbkrMUGiiynV9O8kr$iuE5Q!FY9^a)<>dw0< ztx7jg+#h1EpQP#U2OjN2H!DuSMwYD(X$zn^%C=GIP$!G9od}C`4PC3Wuh>-BJNyu2!1~RFKyA zS;gut3N(Qen2?fxu{15HJ)4^nx*3k!^*NU5!V3J- zkaol-g(z@%XGfo78S&44oIPcKwc6r91IQ~pjQTmGgUFDJBix8H>x zjpoqS9~qAYNc&?=e{yx+hC-y|#uTwl`pf&kM_j%6sSw0y>Bq^7KDR7E#-)cl$s~{r2zuMBT=3YJU zw(C{=X%o!YuqdEMoTBmf>^3^xIaz6q!ST!WmEVF{?z3Bj+u--g1wAagjthqZ2A8kG`Rxna^ZFs4VfO}FL;ZkC1JFfDu2V%3 zdF4aKwmxn?^eql$q$jrR)Vwl&@Pl7JH#h*=9Ep4y1bH-1D_+pc4|NUvsT8ji*g-{T zb03?*$87)eCL*?<>v{g`=Y=On6_#00a}*H#zct;&ZRZ&03C{g1*+23ty7Rg42GMxo zc&lf@#7-GsLYu8k6CsbRNY~bhJa|p=S2<3Yy5=!Tn!e0GR$-tu{Xe=h8`(+%YRi3_ z^4mEw#7r_{G`8vQn|4el`Jheq-V~U=?dkJBy0dglENp*1Ow;wro&x$Ot5|;WB7EFU zVSK|+k<|S$+@w(lRZ$uRAC;UP`n*4G zuwy)No_3bGjdl2S#XG*h*Uy9PimYk3Bk#Oc9V;hjT800oV9pD1WUY&!XY@-Vj@qKo zDPq^O!@>FzdEsj?y0~+|>(6+mfCqPUmly?O?l*9`YWhYcP1s2sYp2vtyYa8Qu-qa( zx;AG1X{M1^7zN8?+;PB=HgXe~8pfL0$c*UBnnl)VNsj7=`TpErQGGq7K@02wG_aDv z6gsC$3pA%cU+fq#p}-XFGoy(33=iEUh5PiSn;xha!ZN>%HXx4Gr*6L4f0`Nti*sQt zbP%Zi%`MvpNn>8Nu^dr&>;{Ed`tT1?*piFa;xO!;&Mhsa_4`Yh1u4A)H!9r2$#))u zRTDH^xkt-LgVLsy-bcqr9?+q^mSuAW`;85kC&~K~(W&R8t~&L9)Vi+%Mk^fYmw?Z& z0C$DhX}6`hxU3;pX&fZ^m?|c+2~qr+ei9r#og|0TfDb}t6IIgpg+D$BSt8~%5gRX8 zcRGPjkcx(c*;(G{9W(ra&=U>vD&^f!4{ZE-1>Bo49-be#Tqd$*K3bQw{%xjJY;6?f zCCvPfv^^~A%%zWhS@ad{^fDsloCc|ACw3>zMK@4cQprWx@}+8Qf%$gzZ{0ruJE8dk zS`2}csz{Cq?fa7)Vq*!8ko*BJrTrjp&2uise&QUJS^uBJUL_z)t!^wjpqX?YL}U#7Ji zy{P74qvpd%Q|QJ+E;xb22G?;Ht!seh{Gi;9(TpxYlX`wPrkjEup-F(aYsh%8T$@0I z5#jz89I*O*5DzBxf?erMnhI4~-}q>b(gpeZYE;n{TztIgcq-2&EDlH{q2DK9q}zWZ zvXC=+X13-OLPJL%x5)6K`t!g2>?mbzmpaM8?8ze*o@(+m^peNvar2fbp~6eLUo?4} zzJpS^~u;KLH5Z{&g&<~}hoja9ir2ensBwO+F&FT4MrFaa&y@F#bj&p8Q> z*z3gwIp02q^~U>iKq1!WUV54huwK^-*%2Df0-zhqGTV|<_uv@_F6&aacMz5nEa65HNKi{H^oi&u^4Y@LF5C2+Kq(nr_>;$c>O8j-H<#cHH0aXDlI1I3Mv7rl&^OfPH@SaT zH{gK>jImV$FO3PQjh@>sNcI0>N}d&1`f6(9fEXzzf)<_FQ8>E3kneQ= zMKF)?vN9yrvrrxEMp7FxEY@>JwvaD>lgM2DRObeHZwTnZDcIST{DkLJMV9%k@9zLF z2EbB1|N0+5CenDSqc?hm^uBZp8Vw>KliO>HXuDT|40(RKGHz@O03GD79<;anw5|^h z1)4V420)kzJ+7XjwfKR(=`^oie{yX8`P=&@13J3CCIP11ryDrT$5rJ9%?YPN$LQJL zUIpY#5#w-{nsR@4**s}jVy?lr-j=Qmw8Rpi0{c{6ODYwQ!)h^wL9y4u8Ci6KU)16T zd`6N3uS1u4B38QrB(!fp@@=LFancGl?XDHNYNtYfYjulutqfur;F8A}48CdUIWnMz z&inwkwgH&ub?*$_z4ND#*Rte;9rJM;jhpU2Sx}$X2@E)iwG$__!mVrr0*B$1nmHr> zNhIIyrZ$+A_x0`>qWt+RfO268V>?vRqSFpS-|(DZCAyr!Xxg6D!8$oyH zr!O*ZIq{6C25DPh0Vi5WJqn$Pb$7+~hUbMH54kzyHD{;{yX)J3XQ^b9;s4IPJP-c7 zav)C|Vkt$_p>H^bjujMBDcSlOwmlmDHy167k6}@%`4Anu6UDNUVR)jLP-=s_L5Gyy z!}Upu%%94Hw$$*C9~bt=-Yp=0-N!bu-qb4>{1js_J4q>N7Q_>_BFs+(kGa+K*58u6 zb_>|4D3tp5*=D_}VX_ZG%7lU5zyX>oCL^0|n7#SGtga)Vf!h4~SLDDLna626fAqT=HtpXY!+(Mj$(DcEk;le#kFb2B zwbB|O*b~lv-z|r~__c+yAV3HkqY05GK}lvRwzHa3g;=@be|k8bO&KR-EPOas$Rxmj?|gjwT<6IK$Db8Q-rbcf`x$qbJ{7Rn zsMojm>$L@EQ_bT4An&W9;_A9Bg9LYX4Q|2RflbY!Q#W*4)libv|5z8D5OqX(d(n z&kt&eVXDSy`I3B<^;hR{?`*^Q zR#v}e(Ue8|1_wvt5*RcSNex5#YG1c5j?WzSgGot*JhB1Pm$bSSM890B`Up{kd#dJi zWeR6WvNOi< z_H4&|*tc%QoovCVrm1tG@`6)f#QDt%VPmxxVk%ctm~j)-l|yCl)%z0){1Yzl}nNL;?2KR@lteFRarmMyeGKuwtIv9fhOR} z?fyuY7<w`(4Qwq~0+pWyb+Biv@0Re5L{wIb1WK?kD#FYPAPj za8DTUp;Rn=z8DlPTR6epO-lA)y0@2I*U{g{zmMBKu6>u4NJ^2){e_hyHCLJ`I8wLYAcVOsPVdfK32( zD<_5O$_8-3xYiN0g-#XaePMqvt2A6IP%f0VJekq?cwXQCLSxKBd3#*4+W*`{cm>#{ zX{ks<7JEN2h;N?fYe%~w|0@}>H~Vn1g4X;96~i^Ibtowf7%EFcdS)%LoBH*d@2hc> z*H5pF1_jMw!yj9Tlhcagdxh}8&FdD@={^7GIyCe;H=@GyCkur3U4LOedyX4L`?V#a zvk)qz8=1%ZjE69qeebPiycQN-Gz~L)b~Z5!)U2DnU077}vaTR4*1SjR;TV;qH~Waq z1y&ZdZHu@wgmS<;fjc{7whBVA(p$0uNj5kb=TK2ZKWSoNl%ZHZ5i_v@g^Rj}<6#Bo zPo8O39!-`w7~U&eFH_5ikg);tX|)eQ`<8JwtW4V%{c zPhUPnNNP4xblk&A$7|r7h9XchWj{zgua<~XBoo~e)4KrZsVHdebkOljHOX9W$qSCD z;D5H*5`HRO+23yWI)2Hf4nO0Cz@P4^%TjtF7W+~`a@+S(@Iw4fy+3r)&*eux>hL@y z+w;O^4&3lHm>e_qAt}jQ7u$Ak1x=Qr4o^JmSw1@D$5^tsi}`cZtE?~BG*Ilw<_c)e zM=v~px2p1#5qOVKMZ+2B&JFawx{<3x&)mkPOD%B)k$g1>yAEh@Bw*u;%D&$u7jVc5 zxk3br$lcUE*jS03WpFF+&BN!9ZbXhAQFvt4K^}fJ&GveUUsGzRVMgf4-_cRA44qeg-}WQ1 zxTHdw$P}ki(Rd)p^%AC!`nXHb;} zsoD=1LEUz|U5tk9^*C#ehH{xUg_04Ua+2a=myHUdeCQs6sVaK0mVssmfn?!#@M42o z9Yt_e*98?T&c+@-{8ziDSAo;*Ps$5dmWdaxk5<;-Ukn+CQ*Yz_Ev|I9MN+EX66oLtfQ&;-wqz(CWubjWuKy-gB}R(y6}?yavMGO{kil@~e4kY$S|m!rhEpO%5oh=ZHo=QY zd-=skfCKYnN(#koZf&}DwJ(Tu$xzcria(#eo<(&je#8ir@enMcu)F6U>oj0OeIh_o zrtx3)UtoB4Ndz<7+QtY;SNR?4FG=ET7UUaODoN;|x$J}m&Q-pb6Q__zcH`a z#|d+oVWegg+k?z?hr0rqI6Al?g;|Xp_WN-KNrZrRr)#%roBz3e?rNN1j=DB<%Vol` zZ}^qH5}{cuc6a&ydIn<9xuBBWZ^hy!JLanv&V;A}>y7ILVu|;JmZa0)4^P4$D>wfj zfo)2H!rLj6oTw}5Z`Fov39NwG<2Pq(dkxcmidj&!L5D=KKYZJ1qBt&s-a zc9Hm`&mRG`dhzeVVn7_`!0r1DQG03JYq)*he3*^kU)yPu=(hDMsDUrQsmW=JqPY^( ztAPoL`x}ZoYwG1$P8bvWPl8Ri5CYM-s1I}5E2_f}Q76xrq-p~cAa_9E`tl^n6&2Ea z#B<^bZpBqOY<$Tzj*B2JL$|dmfUp*+r5q^-;Zlr#o@@t42|NgK*Y-i=>!=A{?A&oG z^AbJetrozy5^6X&qszAd>lEl(@5K`8=U;tduAjsG6_Xxkz;^0IJK$<(__;?H_0#=^ zz-|R7w9^-UZGQNJ{wc=5%c0+DjU}N(#{7 zw~42p3jI*gU7EX$QaRBI@tcIpg<8X+sJA;bk%s*k*O*5!AI^Xj&SMMGIz=3lb$#fT zU#D}Z0v{twCB7Mw{^Zh_TyWCHv1LKgn?;|H^IJxbD`b;K6<@U3>2ps{#VT2Tkyc zxT>`I_s-$|-gq4&PQkYqRcsMAil$P7McpDtQZ5Y8W0Jc>3H_BzPLiot43I`4_Z(Ic zkhXZYM%v%N-J1Q>ik%}%AHDi2pC9W_D0A6Jv17Hpqnm0Ld3*lrA<@Qt(Aq8K4Rh1` zm`zVlbFdsTII7WOlL+~QUtPK6=jV6L&Z?D==LUwyX-dWZg>q5L0ly?ItW(co+7Ed% zV%uU!9N~wjO{ot5f}zTPz>xX>9xvyb6xcL7{Z>mj$YF@YK5Pf9&#VFfw-4$ap--7_ z(guCAWUS;iHK>{-E8=3e3d zf(-2qCB>K8fGbHd7!xmHBek8q8JK<^cbfq!vJ+xJ1tv460V08hp)4{wwz!W^+mlwali7WBv6-Oo z8-|%WiUh-4v&&!q@`rL4h{1^a?E^5pA^-2O@Y~9%1%n0qfA_8QKNAxRRrcgQB*=z@ zp^{Pv3KG-N(S79P1m|lMlWp(p><$hmFpR`iezLNn$H2gFczyA`=l@49jKK;5RxlAg zgr2;-Jmq>TbpV~JBv>!OmzQsqH6J=Q;dBHi9p*UMUkB3Q>znCb{0`|@!p|4JyY9 z&#~l}%#?GN^mBtS>$`V1Zex8KH8a&!1F)RmHoHE4k%Re4O*#6g>0|tJF2)xp{k5Yo z9qv1HulDdbi*OJ>#L>s2pX+gKBb_f9a!Of5p=+H!IWAFWF^XS#cg{))Andbi+d7;@ zK0N7VY%vyg_5%1sPc>ePrM}g@@TE8BsR@NipJ{73%-~co6y91p@ zimo3j-bdMLafNw$7SN_i;&tqPM432+genk6SP#d!PKoDwa%?QTOmfDll2nlOf}Qo*NisQ}sUyB+ZsG+`ec~m6h2{m~ut2c3)xUSC zocWl>HJ~Ip>v|~ZZl?_8bLkP8&&H?d7a!uDU#{JM&RwhR?l+bcKffu>6#HNC%@W@| zJ8PvR9S_v8j>9FkD-s4NtjO%xQ@;ML&jhIsb6w&m@m zo{z%@YUz#H3PGy71+prWnP_4(yPqhyja}T5Fp{!+slPoBj_(R-QwPf9gn;)4zd?Ip zim)qD>%(C2gj*wFZ!Wz?#=+I*k@H%8{pxnI;7D%e!CY{s!d28x$S0K3CTD5*)n=J(1RW1PjC@}y64KZX^d}rBtyuSvBNnkqX-biWDNXm9 zxVFn<&!gWAeim!e(5nbD<7YYO6~?y{dFTBX){~aAFnj9xsVyHHLB0KHPV9dT*c;9~ zsQ-C`VaJWrrssQdn`gOfM@7~V1d38gyhoZJd%!SM@D$*+8Om(;qh{TLOHd$6Hwe=v zSb1=YgyIsmE)=>%8Ta7Xl-uQV@g(TfF7aV?N8vZkS>WU#sK(OOFh{+4TZ>|&x8on) zEOH`i zW^e(46G7INGK(zm6^+@&bAc`H+dP(Q+TlBhL3AHYIt(OWqonmcmwCDJgftyMU9^2b zrSZ%fZi4l@PJmiz5y^*Bt2@<}?0xb*o}0(QUY9GI&u43J>h$zqE9E zT`rl^r<&GuFXa7+gty|PV@OU9AzG`3W(gASv&hN7lF1eLi*ZM?TLlZ<#}Rl?jUGGW zn_;&2GKzXT9(PzLYG*``)_aw4!>WhpP~9^R<})ePnK`Vk1GGj^TT#AFkc?!u>iSTg zKW%f~o%E2$_$CStHXtE!K%KcjKR(c1PUK8`S2ebVYTvJ&p4=EeJ*;pdy>1J}DF1mA z*MaBT&;O7c!6F?sh@H@p6}8}d!a)*E%H$0MC1x4lAEP=&b7YE9S_00QrI%Teg9iDG zE;ok1S$4E;3G5&~5o8BRjsXP`1i*dxdZ!wg1BpAklB{1f8lP7%I~5LN$Qgn-PpMuQ zT_(NCC_|Iq?`S@~95t2W7;D;0X>u6&g+?oR$AQzTfx0Iq3}-F&|eQ zVdp0r2fdj}X}gP1e%1M)_I_dU@q0@u;fjcsIKhW~Rudg1X(rgZ^4rgxRw6A1Eke!(0;R|nr&j$9P z0Sex#uHfB@cNUM4t{$P!PBl?)8rFxtH!z>e_}_uKX4FA(@d;`nAp^fMz2=%HEdU!k zyCZmb=xlMf3&USL(2=IU(NSkO7j4b@yUz|D$)1UckZU3L3 z^Alv`8K0a?`x=X;JfzlOqm4&EFg*C};o(8n@;o6m6~Xa|>2)pM&$(cujX|>l z6d&iqXch-H|FncK+iC>Tow&Fx2Eh>#5k8yD#J@3>hdvO)%5EGM7t?astu|@F?o102 z5E71D>gecPQ2vXc_}7zfje~bbl9Gh?r@I|e*vtR~6!ez)rlNDSb z_#=Z)8?|ACo`~W<4Dd9wJ-xjNyc85};II_YB4>syl)t}!ozZDEMEhc#zQS``F;4TUjWdw#P4Ho8Q2~$OmUI&dyrrPU8anP`EC} z>U_?1C&h3b5-iGtZuvK`aP{kbiwVd`6PhkcydVNFlvroIa60wDRV;~mj7>&kB`^|{ z5|`t~9B$2tGvY?nTWxBxT#aPP^mEgk8|%`S0+7g(1psgN;hr~}7r^5oe-f-<+CVO# zN~Z$N8+B1)kSoE8RgVpCXe05fMmmHJt|AE56m)P6H+GI6Uy%82=A-iKi4Yc}r5mc| zEsycVficAlfY@f4CWK0F~OcDJ*O45 zclVA`O^sS~0Tl(4e2V55YO3x23y~{E`c1f-2Vq{L*`RNjaFeCkjDnp9f|sn;`%sHM z`=Ly_4P9f#Wc0`GU4L9ffBSpsKYe`zaaNzst>o zC@;7XjW)J!-uV`gaSw@tl3B6lr(J{2mW_Y}{(WY|-zB__t%~kJi|x>+yVi*1tZwI` z;vd3#tpkW#B3#P`8N0is3)(-hqr6xb_Ux_EG~zaTTU9LGMQ`p(Th5Tc3)_VNhavJ~ z{0Sq@m_J$lnEr+^wz9S%+9gcmps1}O*A?$yVbT?5&oJ`ow^uZg0OF$wz>Iv$5EhAK zFETqF79=CJ)v||E){zWX>b3hP*QtJeppV6Cd!2+d-7|M2h>!n_pqNqC$xS~FvFl-p zlJ)cOHfk19k)Y|O+(h!kZe)>rc-3iT(7&fPfqp(uJ87Y+$R*uWVc<9Mfi+7En zsgrcSB7$!w6K{438CL91y#f7y$Xu{))c8_D!sdxxjqW!cKCC|pV&$^xmLmu+);2B3 z>ygH2z%X?I$!lxlATzo1gzTo0BqDONbKHBa-!^<0whXgK*DMWE5wn5ui+b z9VkS#6Ib~lD?J$MDvB882wcB`r)LoHW?&zzi+wjl8+JYWwx=VjoLGvB{td-q?K5$( zu!&s|{!F=&p|+1(<0hxQ(0H<9NbMj_)9aUz1LWFGuM3D@ox-@f=GR!ZqpZuGht5yBI_?Tm{MlS*^)l?y82GDb)G0jn-dA!13uMlM1d*e@o!3+bX z;&hwZP40FNORl`C^8xJ$%vjk}b&E@`3na|wA}ZcewuF9(BMe>MKh2N|R|*1*1?WT=f`}$x3Qg z#ODv8dSbaXfUG{b0FYoJX9yXe6ieGzr!$xyim2@k3KC-z(#o$HPIkSt@QeNsr1Ct;?G78@|? z1$mr#Tn8^)eWX5P74gKzm!i&UOqEbU)bj^)KuW%wzAkF@9Mul$mg@tAJH0*$ItBuF zH0${Tqri4#Al(Y#gNzif(NC0fm(OW=^0ldPC`g=wZKolyY2Kf_W2s2INWL^)QK%x8 zw+{3kf%4F>8=r1+oU+`No--vg^+t$@TQo%u{BwZEo>pk%&}t%;jcFruiu7SlyihLf zK#TQ4-d{&_BNL1yQ{3MM#!`EwZEE#043c>+8MIR9<`kvBP3-wj;)CAX00EefcF<9% zusu%$@X%)c1lKko%ZP5P!wYNHz{h+erhlV();82e}uBr zHP6;1_rF5~perKu0qOES+MY5tWimgU(nmO^ex-^ExD=vrU^IUOurM6}f!Qk9W~_1} ziCUa_ycbWk&uY{j?vk)Rt16dp6gaO>1d$)j4ognOyBxzF66@bPu|h{Ftg7Q6s6yW? z!WupC?IwhT^_-gt*<%wQ99ofA8lDAO@C*~W$rTF6TMR|Kc|2@<2Rv1=#U&?Fc zI=4fp3>SZ|jTWpdN4sj$JE>!H79h#qNAd<9I*3a{jE_a}#40gXtJ}+^+W7IId=WaC zbtagOC6-K~no)9`&#NyF9vu>eO#ix@*CENljwQ5SZ!|X#cMiW4X32hL z6$FHkLL-ZfDX{(>r&0GQOj6b?#|kBFB6a8Yc6@T1gW%2mMz8`VB1dIB<<84*`Bfx{ zTyDig*vOwW-kRq_STz-tEJ-VeWVyLC`j~#_+Y)*$E;4Uc-Z8^t&MRiipB!t+X?RY@ zX!SRI^2Y&~Js8;(3v{n;(wHN4^+dx=NVC4{qaAQp2vfU_{WJbc6AZM6R?4=U0Wt}l zXwU!(uKn@74ZOy+7G$L=&~P%jJa+N81v5<;XP0Xn^poF*igLCezf{;DT>Uwt`Zc}f z_ut9vEE3{Z` z1}}r3Q`A*o8P5|5F^M$fXS>#hIbr%Zt&|M6f@zK#b;ILf;5*(Ok~$qmf6}O#5tO0b zAkEUCKjK;KCWdpigQLhwogHlr43IWi&6&WH<&!aY|EYCoQa#iV`-Yq!>n4;?p`P%SNMe7JYBXN z!^6%um-59m-$l;C&b!s{iP&c(Xz`LG=iN>;5`F;8u)rG*@%^=8v`R==O|6^6Ony(6 z7@ZOqULjX?6#me_#Slj4`ko6-F5m8Sn#rlAkcx{Ucp{E@7}Z&{z&+nk(o{{a4i9@Q z-;*9gvi#i5moy!15F7RB)}%57uF9FeT)zm)YaYHUB^x^ZTsQzhpRY^c>-=LptjG7u zBhM)+ydSaMEqmbbrW$5r*fiUE?C>^^U-$_ z?sI=QV*9Z#Y?~aR<8hUtUnOk!?ciHfo6rwMlqm8jcnfjRcri3F++8TT zjL3HUBsC6mB~6nwR8`6-x+s}+0dn#vJdsP&@dZ#P$vEx}Vei6`xG<4Hssi+lQ`C4y zl-rSS`Y?DbKiNNonN@Zjy7p8N&f}nA7tu6eA^7T?uU`zoP&Cnhx`7R`M18k@kJsSr z*Kf3s8eip<1vkFE+3Jby=jJDH(pI(}jI}8vYB!&k?-hqy)t`rNGZ$qQAD|U)6Wd?w zWjt4zpj_^csWqgDvnHnYT%mQwg|?Bl9@zPGZwt*I7jS1ADCzzSXv?~i$3R`rJ#d?A z7c~eAyzYGs+97s+y4iu%dXlmYE=L-@9VbnCVTke@H+kNWcRa1)um}u4f3_#9Hbku+ z3($Ivd5IJDX-9+1UjXkWWa7r zu^QW}2St??rGhodhQ1JNO>FwgzWyVWhtXfD*A7D3OokTx!>sB8+dt6w#r)qL>r@UrdJT9&9w zjzcU{Ob?E|K*=>oqMp|-oumldVYbEG-WEbMa|jh*qIEI@tWWtJI~q9G>qn!5`r+=j*sJ@b3Ne)=fw(#b_bq=O zI79i%9!+6zSv)84y&KR@M`lgm;K5x(?Q9CeR`DsQd&jN!MC&EMe7V3>i$_9Wfy2Ix zrwmKhY02*M7i|~2pU?R+evhRKCHQ41(7oIYIk8%IEVn%#B`4dz2ZK@?N|4wOJxA$( z(;$DCcoq`K!O&Ss-kjc|&3`-*QL99q+iCp}*5zJ($0(f@HV3<#I1%&)6cqTzSTFJ% znru}PsqE{%aXvXdbn<+l=_7Z!^t5U5ro(siCtG9PW8h|eCz@S``dsvV1CaB4559Wfv2mfm>`f>~)b>?dR(LW-6BjlAuO& zSwXSO`&`gOjYr+1vR6fB)lnJNzG!0)R63$JmioxW#s(kWM*+-ret_MVuTWK0J!yCH zR*wlX9Sa_v(YK;Su9^pDYXNyun3H#c*9YbBugv0VX@-9$dPQ38oR(_C5ndJjzJLk4 zf<#eG{?8VLjM7Lg#>cC%JT!jdWw6WLRBeRRpE+=)kkA!ZO^%livUmQ(Bpcr16`~vm zE1YSWrh7vx8R)0Em?Kk{-N2sVs3RSMy|h~=#X~!0wOSF_b8o%&M(m5aaQXu)D-Y#ruGKip7w$hI88!~2i83!Gu>d2eX9-8nkPsogaXJ_Y3C1aW9y%LL=% z63W14L`8T*W=x4bAYC4r!RJ9}^SW1SRJa(Wjlm^7T;jDfh8^)ZVSgi7cb6XY0Sy~* zc^SyDRH9nsaC593931>5EuXDK;m2G=3G5s2UTNaz2R42UFp(eVU9U*Hk zeb$kVB;V0Y5Q9Zu^| z(oQDw8pO`P2&LD6Ga0G@~0mvKA-`dC7tIUFCW7gotynTa}#YhzO6E1U`<0~s0kV6ND((vqC zk?p3bfq1zs)_5AnO-O@u-k_7ckAHaCqJ0GKxS|ac5#rAJ8}`yQTI|!ZojYOgj8_es zMg_Aw_FB~0quVY>g)+#86VLiJhrfNj5c=E}G2;&t?q+iT2_O z%qf2`ClaaUvv^!^^trw?@Ht|(UBELhSE=3$Yj;R?diT0I_S(AaW5J$Wb~A9-EaZ?} zs7IMO_+mbFVuDgjM&(gy8ZGG$)3dd|c4u{>da&Y zFnh*2PQ4X=awQYJKNjDbjcecHawY%q@&aQ=X}$??xtDux%sx7&dcm;uwdan9bz%VL z6x?jz2783$>*|0xY^9EN30YZ$&+-L?;uy-J{ez|cP7Ipk`#LF~DD+`Hz^webbpk?0 zXQ!}Ez14LO?6tHYMA6idvUj&Vo{|(DW$r!(8ks%|xCn1zx5#FxrmC(k7|emH{Kx>% zB?R7Z*)G#k<;(D)D{XEpz}c6r7)5T)IQgGt_1{axhTng|Xu;`ej$KJ3sy6hN?$BS1 zsy)!pbeKI0$STXXJK)|l#)WilB-y07zaEGNrywXp2sGiwWS6Ho)JZD37@;Ys8BH?{ zulwHWyBy6kZOB+r)-#pJ4sdQ$eFmU%-5SE39i14fehPAsDR@x--T*Y@VZ06K1^k2q zn!c#m7fJMEvcIvq>|i3f%|r4Dr$I42+V+<5<5Lm-fwoiJYq$a7X!!nF2-~*!)if(5 zI5u;&YGwRw2B)c&4UQ)FYcKev{>6e8#JZFp&QwTI<^GU(P>|a=*^XSYpqT!foi9r) zCMI^zhIDvAO+kO3e^zsqk=U`)uO7g@J%#HC?4yGYy(A$Nuf%-ISWyE???ixu24vLK zk|b&0Ab7!ywy%4^S=E((V#iFsXIVBF9sl$>e+)&#OU0u!nW1Im>rO+#XlLfpB1AXq z?Oraf`~GiM{kI)ZoGN*iSbsQJ?%$mHWlvDSb0gNz3V(#b4)uh> zHZ^gP-#yX(L)ieWbHM>Pm0FK zBc#T}g3?0^?l7Vr=GA1w4_v#zjPBouuxTm%PTi3n4zVh{{9=rI!y+u9AQR{*=$AZV z!1+79)^C|w2EuKU>BSz#lK(u^{+koL><|C({mC)Z6!Ks+`1X|u_+Y{1zYzl9OPSmh z{0s;9J>O!(^k3eOgcnh6cdGKZZev0A{sh#h9Iwmk#Y(H=L<+wakT~#RzMpW+1VyVM zaNe=ZF~*^tG=$%Pha|Bkc|g4kLT)h>R)Nz_j18PTZQ}<&ny<2oXYqd1kqb)bnGVSV zXY_Lsf%$mELvBhoHVkm!;o=MqJ>j$2NMcA%PL2;!ab;yCpo95`!*!1p!3s8or!(0hDNK~`pnQ1uuvWAfL4;nhDE%lRU! zGO%(&GJR68pR3MB%$*#oA4jqOGj5T;5fz2Y($m)`F65Ae@+YGB%a2&7f-@I2H7o*d zJ3X7AbKG`ct?3R+t>SCqy$(%fI!~dA+b4LqXI~}-bzsa(FM+6V4v{y*bW%)hmuP#UfEyxYfZ-gXN@ z3bv1t(3x{PIJ1c%kRIT4cPB!CuKUEUG-h09QJQq*R_ejjms4IMEjJ%jffaTOjA5nx zQ9ZW@qeFs+I?0zk0}gDb)-CMMR`4^55)h`4$| zoff4lzaY{(tH0&YDj*64+wTu-wUy?s3rt%+ zHu-o#Wi06CG`FF0qnjlM93WhSW*8CGq&}t#MSZH*aNqL1gD_gd@JCV{oEELV>yj&@ zZ20ccg@}T0Z|XQUbKu>C;-+9hAxnB`*n*pL&9>FPLvpH761L%IiOPQ7*4)uh2M4`I z8@K0rT1rYN`$+3cGQT~!#(Z$wiF0A?`!}u21}|LI(&7HX48US&@RDwAX_@mmg$E%`z%LDpes4pa7W5VedySbbgV(bk^BT_@hvwj+y+lx7a$ z+v%$1K0_e(*{%0E5?e9`E#OLUYMvP;(f>wbJ2wk$d5YwDowaAI9?`Mo+~umY1M|pj zZV(i~B)&qqC~z%Qrm00qnRGI&4bo%V&QU;g?8sZxXw|w~vWFo$Hr7@z871#4p=8#6 zi4r=wMr&KdfzR^Fu8mNki&%_C);$8Zf$OSZ#hrWzyWshgA9hPqw0&@HcQ9j)D&VHu zb#tjfVZIu6vQf|jTxdjZ^W=~1r(Z!MxFrbcX!b+e*<#)>8ihnfPX+X&^YgWT7rL4no?Cy|liJJqRxB_NC# zx70J%RJB(D!C9rtp1 zZ~(&&Pv%G$(<_;TBm4-fO}`>*@oMF`Ii3-NAd?}c#xy|GwiKF;|Dr!f}fo7HiPEAB)G?G+#!)X+7Nor`e z3?D?ED^0!)+ZQ)#Kgz3qSZ|8~jKoVnEx>|DByR~_BApf#(%WXO+8K;Ka`a~i4#wl) zhL2E}8^=qbRcO39y6lTO+JaB)XA~a&Up#ysg>CLVnN$EKbrzASDoEP3Fg5ED0J*R6 zE|H+o*uX~=hLLIJ&Vtk^L8c+EJt_lsGjIF+e)W?{OdLco4=g~M z8fi7cT*WYa>QuBxa<0Rj`4H`N6eVD)B_Syp%2s4C32!f%5iqgmJc-@iI{Wz8oUR+7 zx;xl?Q?$YAF>K@t%7T3!Hpwb4G_9JwDdN2ATS76H?&_z6t(tp0ce!Y&>A^%&I?04f z-Mxp?>m-}_Q{F>(AY3Zkz|$Cfx@-!HO&oSnS`#)nO?*j4aZyzvSC_-cQ<awhGNt`ff!RgjJI;o7r0U1JjX%o_}`2v3CNRB$SRS`&pICMV%ZE$ z%v@!DyxggkX;mGWhf6bYstXd7%-S{<%WUknx}SCYusc_4kq)YpNn(>(n+6B*e+D|z z!6j8-R%+K4s_3*y5jo+h)jT8@N-QjDTPJ&Ay_G!$SAyCx9iP;&HA zE9kF{m)a;nIrGkAtTlvB@F>qm1#f&DNkmJBcDz|Kw;_lrHJpcK_X&RWV~zMvo#9`B zcDkwE&>f#>1BYF^!=OGZkJqtHH`opOoyj$M zo^vR`CPY7Xb#>#8JK_~Kb*^mK2?Pg*K4{7FKK6RbwQ zO+yV#2^jE6@NTM^MbF{(ML!^=GFjfUn1X;4gh=LA6j7=|*9jk-OuIIEQ83Lvc_09% zB%`C_-fVrg;N-B@Mx7(%3X-@SRBabvvEvGq2|;ww^22_4+*U3~L0Fa_ow|Cl@l>(~SNoVM!q0cRkoe47)7N7UH2;813ZE8BUFkcJyZ{c! zbAgfVe`Ih7IZ)*;1Oey-|DCMyy%3jxQK3cBCx|Xou~rEobK^gqBhMfII7boq^6g)S z`mW3Q&-2`9%78_lqR&5_3@#P!|EU1T{iHEz-A6~NEVQ`lPu^D97=|1KN>Us)UpeiH z<32JTQVn$Q@*917n^Ex5-xck_nwS}w54&8fm1A@(TR44`0`^BlK@ZN90&b+@9)Vh1 zzryS^pjaL5MC#%n);bCeXu%#33o#gZi?cAojqthl)YJ}}#LI~dr&fN)%4HM^Q=4!M zx9l;qcb&!RYs5V(L z-KWD!iqi>iT;xXGFsT*3GNJIW9-!$`Ovh4-0zbGSFk%W z_;1&%KynqId$g`zk_Mf_9Vm#!o)Jg6&Dq2v`DnUrlXujNHrizg?AL zweKih)IPpQdmA2`oAIVRO@1iLdT9YU>^GDOmZ}sIwi3Xn<;j6X$@$yjW$*C!ojY3JOthMRG-4UdI$uI|`bX!>G zafR^8tzs(`)hD5{|2abhJtxHB#|_wvJw%5SKo1Y^;gtdbrGvVd5R`BX5^6#C zQ-pRwK+~j76}4oFIOEYK({Lf@clC)d3?Wp}R|EzF9Z|R0X&x*!8<8iT;$6uw6AHh!$4qw!wReF!c(!Q0=*LVxs8{$M>-#Eu#2y1@!FQTs- zhSYgbKQ}M^#nWhk$hL75#vLPGoAy}!}_|A#%pMdADzWwGZgU4+INn(e3j1Lrb@pq z`y;o_r``dJl^W`+f|;CDSLQTFigh?9MuJ}H*s=n~O{UWXn1a+^s z4Se7IWqq50I?@?UP$>=VIjUEr@kx>`zS=PaoPWGgOZzEiUkR>O>LedhqCVOf;9~D` zbG+vcWG0iWWM1Y_aRQ$go0my6D0i8m{$1}^KXV3ZXUKY{!^w1Co34YF-SPa2p3gTz z(tejsLDvHw8@@=U2QCuWWiAx!uCvyz*vO{&e>8ixW*NfsSAlrxir8>WgtN&9y9oU$wa$sX{nzNWL+ zD!9Jxh96SvzolGc4p^13*pCR4pni($dQ^fVN#zz}T01^F@;7FkEMG#Ks}-6cn7B1u z)IV=;2?~w|TBOzstJ&;&Kh_|wfXuAipC)Tj>gs;w8PD)}STDNct)1y#mH9#)-JVKa zH}-EVgXX5zib=um=aHU)tLY~U;D-t1YdW049awEWIyDk#H`n#$O9q3*G zeff}(`Al*Xues@Zd)+BWh#t>V)e-I-Z#!A;8rvGBaTW_NN0S}nk|&Q|)+(PG7Ji## z+zDVcvTvv9v}s8*WT6-z6se&p*FX#>RaDtv{3=Yy znsdSR*_AK+OEdL=%5Ut29!xV|8lk2e79@EmYKvWN;9P*-#!l+M1!+DZ|vQ8k!Gj|WMamg$quD#8z}cS0phq!&V`U#v%c4;usJtZ7A) z*u1X=ygObd%(e5r{P^;4%XhUx_qeF%qmJ_QUO2{jLA4^7tk#p%=VFNFU``wny~qdc z`v>~etM0{@F(u`vg|b(z==zhVFYYdf&+DY$JbK;kl*(R*yv@U#uF@Akl7^ze%DPux zoeZgJSR3(G?Te0#B<6y1uhfa5jax^)co!O^2K^BB=RWFurd1w7E92PlE}H8I2@Z~1 zxKo8CyfqIxcpSBogFr`+Av*+=`7vy?&E%&S{!*yRLR$ycr`_8GdIw1X7pTOe*a6)4 zoz7aUKfXN*HUTCnZko>xcxcN-H8mccug;7sk8(i!{N$57=zGH zG;R(5NgsJgI(eeNJ$85R2FCqOAZcPs+9PKtD%xA>u@$$s!DELiy1@TRc?d$ymA7|Y z@q7e0Qm8mup1cB_bcs+B5c^5zz?luzoh>IlzT4`1d9s3^37%n)2)Vj{2x;e-)bJ>P zdWzguJ}%B9+I0hO$-Tm8I`rMKP*m=v3_!7Jjo1H6RtUi$#gcw#*fW=pRa(9NwfA19 z3yXl;G0sHed*7mTHMZ&I#)O)2A!k8}l;YZz$GhfjicN}T)P^OPq*~b(eh7z}+nz>t z0akOncNeQnqAj@hn~JE$cTZ$7Zq1!suLZ4J4ZQm=q1|I{utJ2sNZ@iN$ZBOvz^#~r z0DctdpN07A^}ZhWl*QBNH9U5OHkf=Pth5!$)0g$rT3PXdT*)^m}3(UIo1DG%RQ$*WQt1J z`-p6zNF03e&_et50>|Y+clc-syz#xE%vQoZsE)5sI&D`^ZSf5GA~#nnLQn;RzGGq5 z`6)tw-|c(UUxU=&;P^Cs{|;SaA1i9J3i^P<0v7SSS)@6#_*LPn)+`5LU;z`D7Y6sk zc7rRZqpo~8X1B~v_Q%5qDe2$p`I`2jjf85!8rsCkZFAU(F$`j_;EH55@w`}^k0sft zzIo;r3reIZtmwf1LE1Y<*Y)-9+D*g8Xv`)}(%810G`1Vtw%yqFj%~BCZQJJA`96Nm z@0@Yo@$UbVG1kUhbIs3nU$oS}|3q^qks z`HYSN)UgWp-TwvWS$Y`(i5X3z3>TylTpyYsmNAxPX2+8S(gHowZSohx^1i+~B3g5m z+yO4hobU;~K85hhBY(z$9>U+!Lf+Sy3hKsD-y{mQR+~vTNi$I_!Af`v^y2pI!);2V zQr)NrOC-F^LzS@`hFDOQ1ZL0WnV)VuYhWf>=p!b9R}XCTOOTCpMQDGwRhq1bkw?aUUHXvy;p4@ zNnt5{d5{(5aQqtCOjWgJK~Vo9i#?D2a$f{&jNyo<(sKvg((e>?QUT{fsm2evaGoja=+h(RI`Z6p3rgpgJ`08n3*^Jq zoZ8D=g0tu39Xx-bH#j%UzUd2v>m&zaFs8H1di5g8lBQPS=lS!Smp-kYOI6bsmJ4uU z`(ECP+KB@xWsl*{$e8d^ZVnriWi)MTL{9HOVUAan1F=$Y$q0Sz;#nx5QMGWf-* zJEWJLHqOjvBprdb3^VU%9|t{O7a{D9fB8KQ#6u%dYq!NjMq({~CF8Yxn^dtcDWlxW z!@OB<4c><8LsmCwIr_6lT6HPOh9z(fVmD!NuCU-v2T9@dOqAnpS8Jjqzuk0&h#y9J z#*=J|DaPX&{%|LSxUM1DGyem%`&MVN9dsy#Y?zI!OiNbS)#6Cgiz~kD%W#9Yx5(n| zfZqA{kX#1+nVE+HI)ExDJA$A!Stir`*D&YFl?B0EyEJFJ<=YcNBST_2^g}AMYTvL^ zD5cDMBt0Z2Wllwh(QzN76#G+R$eE@|^5+KubZnw@izwer^|^ZGtqUpHxa4p1EGEmy z!VMmq&AJ?Scrg7IZBNE?gFAo5H8eP5bi$8PGVPGk?vQ;f`Z({Z5`q^P^mA>;p$-oQ zwIjlfTHlzzk)S2JGW|Rj`uyD{@A@IVDPlp8;B~89lf%4q>koW zr(XD@fD|z?QE>?oHGbqNyT>!RJP$H|WG@`)L}L(s51W9EEh11t8?9#qS_>8MY!r zz00mevu}%MLFvfNm@Vy(z>bAy9#v6gA-*?Kz%u4`px2G_(KP)y-;ZW8&J_|2w_-_mgskyr zyPEmPSP*O8<8XA3D+2o@SVr5Kh%oQFS{OW|Fxpg+g1_aUKh7fsT*gX^zflW`7DCxt zVWX&`r7!PP?@Lkeacel=WUTWioqIS_BOc3K2ZqsDYrh0?CUR=0F08XpbyPi=Wh@gb z$k=dwN7}E)MSUi|J9WC$_c-@loBy3!-&@OEm1KGxVx+F4W+m_FlZKv#`eY~UPMV#yI@W>-2 zngL!)wxShsQZqF@(8r(SGg#NHcRdDjGD(3}z5YIPNlgs|4HII5S1%xg!3SU3y!d~z ziAo*|lRHHH@v-O5P}0UaV06KbKzZ4~eho*`=hL|Eip*`uS~mS8Jd9ZiVthv*#5QT> z%gsjrOW%z`;8?k#cSF|kV7i!_oCIvE6HVDXnx3jJ9H%mI+Vx4Sc>mW_7xnWq{Ty0) z4ZZ&2XD8QLTZmQgFWm-G&0S{?N+;7N+*l})9s~E^G_JkbZSm{z8Uz~n8)5pcje3DC z46_&z=uSGX1-A=!Z)ooCOMbsKT#pY94|ThI!LOXfrgCdFT8(qhpxjhaar>2mwg;Iw zThIfSQB8Zk#>;NP>DQ(iMd?QB*`|hiatU7GRpNW1meOjFr!Lm=>N6iCbVMwKscM#A zqXcyBdKNJqJHfskI;qqug@Wha#nL*Lv8Y1$LY(5kHfH2irRUb*S8LIZ$xI}#uWGsLz5q4+z(=4`v_vyW!hY2<{DBUVK#f7Bs zLWcfD$93HL%!}5t-JdtkctD~+SoZ4d^R0we>3jd@qiICmrTo|n&SDprWDu;|;{ZiG zd>xH|Ve5s|>jpYZzBk_VLRhokb}h32Et1cgx9`%1Bn`a_05XVJL)_L~Utw zlGV)>TzcU{IWFdLdt&k5H4vKpe^hy%l>Nv%oYlxMxoxwpN%0sKdHweR_7$W3G_JJ@th1U&hKZ=x@w1s^CtA!8VSTVt6 zUd|LbJyaruWeE)MU`l&mkd3qIjVwO4R(pD6E&=V4T3cWr2S4)?ovTvw%Pn;xg|q^lvd2M@Pgeh!Yh;{IE-q@75F(BnqYrWlHckowF##sEb-PG!(SZLzJ~H8TZXibu&4XE_v%a(a0z3m%+! zs%3Yp4Wq~6iD^c6~yM`=r zsJ`#Ru!!O^<57);gw1&eE;)KtR8ZXgt=10s>^Gj>+aKek7BpI}?Z*R(seQI%7C$e) zdlcrMJ9=8aGRPP_zd6sQ5GgBGUGX&kMk{G{{S{&Q@>~yad0&x(*5(PgeHIs@7&2=y z-QzhT0*gOtHr)<>1-$Uw%CNrzaf3TO4Cc1*c_miMlKLo*$LDU~%po3{y^KU>ahz)7 z-H)odZTib9cRJ{MfkdT+Bg>$20uxrK%boW!hM5XNI%ox8o7V2r27dO9zWJ{pgO0k+ zL#wC3d1nNbc=%<-Yu#5#P`1PW0Usq;r~dP4wV+Yg)PY9QSsQ@uZ4eDwb%kwq>-h~w zOIzA|p2_7~P(v0M*dS+ygSM5xfs9&UwNZ!g5XU)z!J~n(RC@WC?J%iVgzTmWi?PtF z_w%XxHcN{eI7T#Z^y3{SVlMQ7An zEil|CmVt5dM;(if6a}o5P1K}=MF~IfZN8yVND^J$I)RYlA zdq&}wr3f|o7D#V}RuE}6Fo{)ZojejmZ?xUj7-r->TI~21lefk0zw`@z4%CgEdW3+`1zuFAK zwdkdEIsW9>!MBw+iH(l-sWBQyYkht)lV(TiTvyK7^24+t%!wKKg?MAQBG`ZResNH) z6tJ^dRwlaq3nX2Zm; z+y`wZ`=xMx{R_LIh7C{VvqO7hYGcC@eeJwtkSEfwHK2)7)aPM+d2@5q=agGH`!x{t zsn)>BhTKKmjJet6dcMhyPPXd<(<8sM&*rhkgjFXC3&>$6A4`V!Yqj(hR}c+Px$oYH z9tX;qv}v`i23Y9p64XgBAmmgQ=yRm#`Mj! z&aNke9x$IRY)>v+|3<_7ysqyv^wsx3L?A;X*uoNN*)ww7YKJad6|0nz6SAtJuGe*v z19dAs3FZ1ga`i;pclWFyAk?P5TBma(f~8quR%<{p8hP3&LGW& z)Ej$DS^cdxl_o~sz);{tlR7>tVqmYEtMR5D!f?tTCE zwtc4%m{99jVYPtMiSKsL zwklhNV@8LUF|$lub?PLA<@C0VT?}FP>U!9oTcT|-Vj?l!Alh`wob)6fTu%qlPC|EW zhA>#aTBbnJ9Cw|hYU|A2P&#_lfCk?PG|vur-;*3juK95%bly_&bl$s@*OnJ_saU4o zd+edm1Q66oqa3XMMZydO-%?x@5)_K^H8caXJYaihHV3H895T(QFffis|cGmrT0ovJ2{Dv_N z*6dXN?Z+tCs6*~d6mXgzA=Js~Y<1&Z-C%B4ATf>J3-_F+^=i`sF9FRSj_{QphE!8$x`{rP`L!;PG$(D3BVJmY1S zKA|V{r&^1(qne=D+g$aMnziY&bd*3%#e?W=VddVwj7K|WLzDt-|1DnVja6UTowx%} z)3oUp9tFOat^|Sgb~#;>znr`{FS)3;ed!52+Tb{rm1`S-T-cg~tv~7$p-y)WzQ={s z8=iv@ec-x~ZMc>9dg?T2VoX;W!s;5+C-6-DeGS(6aX=S(YYBSUc1RVRga^g_C5U6s zb}>QQl%j!J;(t5fn@eJSbIP8tpqYDrWQIs<1bHM3CU=!9Zg;on)KEKI_O7 z$dP*je4yWpv_#VR0XL~gn3y3zhSpJ@l)NkN>>w~O@QNSI}V&!q%`L8J7}(7Nl~za0rS+E`LEda)bF!hf$g+`c3tY^}2Vi~K-@ zX|NfzoHl;bT5rY!mnvURuWq@6`hZR>>kO7 zJUY4P#pb5TlK6$>8M7DOVr|52p>dZRp)0w~~L!|i=`_Pk%3@H!r_XkrH z6!Z@s4ml87X|fZ^&Edy~Shi=1ZK?pemV0(PrjTt097 zePf3LORvU7cRs+cVJ3gmi&Ck=sb6#VNu;Rxr-0>he7Z)7?d|XS0y1HT7q5=u*7QWc z8)w>chhhRuBz{Tc+87;vC2w8-sc$n-UOek6!b7)RcolVPQTj~y>EBzKU_SXPXDJK9RKolAdn zXfbU;ZZ;S##9PXLI+3|e&2ZI=8dCF&Tn4P7I2FH%C4@2UEEE=s!7IKp*c1;vOSS?H zuT?-5 z0X;9~j%p6op9Ut%)PhFu_GOo|SpZ~xY?yaXh_~>KdaLG>b7!=*CuKj?%f)yPx*aZ3 zs0>!hbf^qksQ$l1e%{JP)_k+?5gv}`|14C$nZM0*G+_Al&l~CVIPvIIT*I0#65Jj# z^IvP>>(2ZhkXOEi8B?4%vcNFMhK-74p<8prUa4*lc*lee+I71;jf1k!nEiHMY7Y#G z2jBM7NCXVr>#5sG6R8ysi8b_K0|Pr>SVwMsduFeGqa081TIs`8u8zrbW+I@CEx=@Q z*!I;(w0U#I!slL{h&iw4JgmIM0PQ(Huz{i;;Fs;S6;OUyAy{922yUlUWA-HB=oU+K zXNK@+&7kf3pE;&<=uqHp4EO@lT3O)_W?2gBBAkUk@XgOU5ZC6HdkXR(HgBeYE6rca z4KQx69p^IottkTQu+MP;%Da^A9|8TWr-j{$uc#y@I`uXKh#q^NOsU|Mpd~P> z&h;Gy$zZ^pJW5EL+Xmf!Bf%Crac#N?v?OQNdepH77z7V@dq%5t@KbS!|$8_17J zHP=UIMisSgtr0av)S3=UNZkCQEUj4`3%R{!THn3yB}>Qu$RKDB1#|T0sA<4jYJr-L zU_+)lLh!EGI7{*P#fcX;Dra+<@RQN-zyJm$o8RJwTw4jypo~$w&cd&i&D7Y8QaGIA6|qP--#ZrL+|rg0!f8DE~%XqHZBuzX%) zknU{oW$vygCTiA|dbu;2U-5hD{Y07O4E|=Kr?qsJrd3HF;k!Tn&3h8peEt2?!4#~Q zq%p**#&YPRo38aUHR4*dcX>HoK}*Pf4E@&oCkI~?8+K{P%!*n*t4Ze5DOt*wTk0u$ z`RF?(i5JFJB>Vi=!2o+6haj$GsAHhJBFb_(VcE>gw=&Z&8sLP!7Q}R?+?@H8XU6K$ zMk#E08uhmIi1|Sh!uA&bD{b^YIlK?pt1?Sd;~(EgnX7E_h&?L7Y$RzOoA%*0~}^j!YTgMsgwo=9)Q;N{GNg4lJu6_j)n|7DB{{xivakOfyhb)j40X446$`+4vq z$GuI1j50AIf`4)tds4*OW{8v_VW-v0O?{Lo$n|YPj#le8)oU)oi?UnPmD5ttL&bu% zKfL=Dz3oi06)`1gKF;M8Qef**cYJnQQrB(U(U2coemN4Lg*hKZdWz5O@oJaNW=nye z^pJznBQgKH_GXp39QrHRF*YF;wV3K)>Q{S-u6sX71Y5Xn?fh=&1egZ(e-CfNJA@Jj zx#>1)W;095Vxq3m3J<&7VeI`NX(%RyV)^{}uBXBmHBiEUfywuAKTG|MS&ycGT;ef= zf5Vn%Cgqj`;{Aq$QjMa|SN2&9qje7Z-01}&h&fFW9gC4~KN+vmE(Po~Vtqp){6~ui zntB@oXGWxkU{3QbiaiCV;7z*ZZFjm`Fu>|l#EL6($Typ!wN)?^7W&D%XM}3-O_N0F z?(3cHNraqtxZcCN)hg#`IJBrCDzl7m0X29p^yHCIIzFX#=12BQQOY()orkPlUU z;!F>5X;M-$K7*m-gAa=q5?S}{TT2;-soCu7aE>75t_?k#+K@41C?k%AJ9p*zpZOPt zfwurVwl`paWqeN0e#{E@I9+Rn2zH_*0cJK;%E9O zIdSEMsIX6GwWW!9WrVZF$mpE!;3H~xklvtJ^_U;ItLWg@d+i4Ix}pCrpZ&@{N}9dy z#D&yR&*Ltz&fj+f_A4|h_)o9)WK*ee%K^d7@hq>}gOCEwElg?3(dLw?I@nsBKX-Su za|e3q&hekjo;DrU#UKKJDmaoPXd6>Gk{2Nno-LF|M?2`XFt0)$^ODCT1WJo24J+;& zk5!LRh?Mn#2+Ux%{lz!;W&)&9>@Wfr0m_b9Jf1(QmiW%|bbLK-PSHS0*7JyHZoIDS zi}m*R_7q#t=2MDKo4dNlrNHPh1JT+z#yrAFU&9&iHT-V0qaq9_8rgigM6%^@>cPvn z&cdHAm#^-7IDMfTlKw)UB1XtJ>;@2Y97r4H3*y~`UrxTkY9WTr93P677W*BYH+)3s z7wHfSr#jcy@u}2md7Er^F}zXQAo8~P$gthdR@iDyV$0;o$$$~iHfX-#`pui?SDpEv zFoRIN{LjC_?GOE5sQbpw{+zA$^@X)Fe{8UFZFJw(C$Liat@m`3uH5mhU|{X#@g z&=iULnr>zc+@`EwZ!uDz)lpKCBSE*+0D0vkP_JFaf9h(NLR6TOix9C;I?a_cJ&d`0x#7b{fAYQmFkze!4S-$MwclRjP z#xGetz_ia_nWWp|wsWxjSau8;XxI79sc>X_o%YNxD3j>vh-vs{63Vum&~;X3k7Jke zsYQ0{eCJ*(n4zJk+QLrfXQyI1KDb~##KcV77g5Omg8 zKJ9|@TM(zeJhzJ0a~Um{r=}-ptC8xYlR9mv&V~K+I}p>n2|UneLCb|y&sJ)saeUBF zJhF2g^Z1ri_G!H9;(&28`o2A)0grUBp)-=&#U}z~p`*Dh|bvHw_*g^Du0FFtgPx>zcTx;E0g=b+*9Cgwj}D5x~J!=IHh2}?A|mqLw2e- z(Yd*Eo8vinjTff6n>Z0;=#r5bFP42FPe6g}n5EKL{E7c1k-v9yDz-cWK|uUA;b5te zuy*LEbB9WTkv!8IX!tfsw@3Td;4m>bh0NseiK^%6U`h_>B}luMIA4r;yopBe29_Vb zyKI}_CeE@vcN)jfaI`B1Zp-TY|D=)xpbrHXW_Aa5v~0)1Z$*hx_ebJB_bO=65@iT{ zL}xX^!pcN+7Al1<WRXjI#Pdi!rumznDY@#q+8SrOf+`lE}VU2>Ii+MAQ?Sls8`BHMdBLRKfm~dp2r0W;|0qWZq`?W&#a1I+d!()Io=96} z!Y8d~so;h!jO@gaq2nnB&zeV~vMAqrAv1|kNxT2Nc3@0dk;(C&2YZo=saavMq}-~< za4nCTNH)A|P$6dFbJsi~O95Dwwoe)-Z<-tZ|1G(CSPk<#-&1^8e7vlckF>b`*`}z8 z!R@MG##qWT*8d)7(DN~!ODs46gxaWKG;n06Jy0wbl_9d=`u#~E{ZZ$lfO$_DdMTht8G#)! zY;S!b_!PbupldZ~U@{=x)1u2lneh&yCzXa{eziq-?z9PahIS(eSKeg=N} zB#$g6$ftnl1E^HBg{24z5ytKF#8dx_{n(#yt5ai-X?`Sr9Vg|AMV}3h8=X z$-;-@%1K7WC66-#3~kts}%Ah^Z*#*^U zv9YlzH@CMyMXs$)S9oIqtcKls$aP)WWqJNXOZg#CUq!VlD__A4Fc}~`-p^KZf^n#{ zWW(u!oaNP1pa-<7YL^Rq_^Uo6?}CFWcnY+QE?K1zUhY}sX+#Qcb#Pl}?!ewB8(GsZ zDVjX*I9}_m6_TBuutnB3@@g=fz=gTD{F)KiZFap%Q<_Vpx~BG3mJA$dc<)Kboab&I z0SL*@%pgZGnGqbPi#FM#i47#`h0Qp=k>uqKguRx9-g$6$H*Q81;m&KoMMX2i0822vzrxl*j`7ac~L*VNuBv1gOO_!Pbe;WAjT28JFLl-|!8CdkTzliR;GQ`{xV< z-7bGWmU5A1*l7nJVEUW)%qYImzUe@$)Q|8s@V_mqY=k$)!L6^sHYL1@bXUpD5ncvAfNxM((*y*}XoZ;!=twm|Emp#Y*r~AnzH;S~dfr!n z_jguW!X02M`+H`tVeA>x6o_H$&Y{b2pwp~I@udETK*}sb-jifsX+z=Brt>{KFcjUKsmFC`}*Qva?~WvT_XC7sKh`h#@^ zO|`Nm0e@j}p2iQ9Yc(EUaVU4JuceWCG>n&nv7)5!JL*tRo(~-fRZP8^-}>7D!ZL|m zz<;t~C9t2B^D+Bh=@HfUk~UW6Nq0)!^N)5X^o*TcUi&(32A-@aKjg|L=s_jQQ3c8yZXsF4HSn}xCofx3I$T+%Y#aRFD zYgg#7VD5!ihwHkJ8I1kkXR=w`IDbErJq`CdQ|)R_|51J5%R+PBdsLjW@(1;-Ts)MO zhKlm1#j;`JILfk?n=~0?e=2jQ0fKz3d*P={s{iybW%`fFhkY}4_kdt_zX!czAMZ?n zlVfotVm!Y^5{7f(bWn`!$oS>E2bWuEe5U2jL5#sS6q~F%%Qr%A#X54)pHVAbM1*?8 z+~>_4{a`bu<<7){M}oUk;9~6W*%P~|(|`?=_}r2OV2u*_A9-LQEIBc;Q2g)T zS)swfKanQGn(X!?IoqHyF);zdi8K@vH1egU`*D!~vKGB~1F5EY%~g8-*~?>(9Oe#( zfIKh7>r%nC0U<@Xz~+M$D_lifFOEX3pXu^!4ktynPco`AU+d(V^0&Xvj{PuD4PIpu znzb1kM31k)pHM^J9koyP;rrtFNhm{{$>?s%kJ@1d(`CzITT~vC9%7; zmEW92Ipiy}2;c(H8puuutWuLFt}d#T__hizu4$-=;450#FBY+5l~GFML;~i*s<&;1 zstfj4&(k*4TRD%_pDVKHjkxE*Rc#}?yL)qv zMGYPl^#Mi4u`ldtJByp{<9Q-{%uAxq$vpK%qBPX60-~1}Ho>HN zXL~s3cy|UB*QJ)FDAeYq}taDM<2V z-p|)Y6LYuVEj1zw-vi>=FGg@MEo9;S1<`RG3J$wOy--A(-&!S({b$7jw3KNc(M}N! zVV-YZ2`df;NV;pp;%{^l*=8#{mr0vp9}X-gWgx4t2(Gh_A~UWxL3gaB|T+LWN%FN zR$Z%p#1}|Yy~teqC=CaXQM_%P#6#bztqt3bR))=sC)>}hdFnSm;h83P1t^k;pStu% z#T#8AK=;~!K#$Ua8;s9@`?oElx)jX|T_=X`=@;I|;>zX-$oR2%9uv|g8P11a-DR}p zgbQc+hVP#{`QOPMo635JM!6dmfq0QBS?*e>Qlj+p)=YkBm3d}84X~N{KLDUdssk1f zidXXPF$*F?m9`VnJvzf)ZEDLz@n-rdhq?cuylx7PFr+AJYkRgQP@!DoW}lnkuey^u zxm2dCobB`TD5WwrK6UkGWmu#(_`J!C@eHo8e|ppYN$~t_X0#FR+^q-(oa)HGYYI(C zVEW}5)7h2XnbL3VS9ZMfmmWQUI;P2meoY3Bh?~+`u3Ax&0v}*>qt1AwX%PNik_VhT zncMp!7_vR^H+Z7VgubKXrh&$X#`w(#m!R&}?T6T?`gPTM*X`Tiq(1HBOR1sF9dNav zWN?EmTYyn>My3wQB@B%7f%pA$8uWuG<7*gHZc34A9ieH;tYBX)V?+y69PEfj396JZ zCrEY&N4J?3tXy2oO7FQt7a&SwbJrS{PCVJ&2o2q+^v(I3(yjJq{KJIptD-|Qo3{r= zoEZA!SIrIdd`s+(dE8@9o&m{6m<^y0EDo*$sc?#bfZ)?;3X@-?^k#UZq@a)tI~puJ zd|*IdSOmA`ir`6F;JC7eP|D)7^^?I@tYZpPEa1ad$p}}y?XI|ibi{mc#~yuox7D3= z?+q>p_V5Yj#RnYo*T9_&y!z8gWtgB*!&*^7-O4+;+6&t!t6$jMyUg&5ry@)Zo6hoN ze_{?AkT@LYyyN{)P3;Eb_y4lf#j}6F0Ai$vzo>ytbyH5Ne-pt0crh=*w9OZc!Vjq_ zF6@GGtyR#I%!F6|{P_ZGIf(*>MjCbY)XJ?%>LnDy4l zI{9;zj>7Ag^w<cO8k4Jb5UAio&QnU!Ru@TSEi2$jn)sz_UR+G#Jf7 z*ZG=p%5}WY>3r9t;(BIghSTptd6r3Y2n<~qap-i9n%JI{QxkT=Ed-TZV&2)y6FQ_p zKH4w~a@Fya);I%IPXldhVhuJg9nyaNE8ok>VP!el61zr#=k}Fo3kE{ZxJXZPz*EiF znqf@!hNJ9yRTqUCd}xVSl?SQS@HiVH9Ro{(hV1>14uI;vG;=pko51f_AI#rdBvGq@ zxw<#3AEjowO`88W3`9b$o=SVp3_fFtCtcoCZsMI^}c&=z~I8iNEs-;8xj^d z@2dqb%u??FfgPvnjq)NTLnVd;ey(n~(^*I?tl8}pE>m3 z*@!FK+(5WvH8z~M1llNzbpuqFS1%Ps3{(q(*49V+CK^(54n;sOyYiQ9l~#&EP)nO7 zb$(ROShsA$f2Sv(^@3}6LJL> zXH$wRL!$Dg;_x4rVJ^)1@6k%4gRq>XHbH_GQpBqw{(WkKYB!jqFx4Ks9y-%$`|!)O zNJm#AYppM}nT!7|A20Qc1mI3u1u#t|uvZVM z*|h_%A!t`|v9n0sgi~c!E22AHe$pKt*oW&{4`dNwosl`K>E9QLyoX5{$v7{O;G$pT z>h+YcaB#c&ydEB8+V`%Z3v6)ZPwSg4_vGZn<3YIv@?F0+_NbK~(oDXjF5*Tz*e5ym zqMq$j^w7>gol{pK?^a#Bx?|fXh6^l1W3ujVgmYv{@z19m-O;JzZmK&WKh??8NUTok zb4MRpZT((l*2zeUS_+;uTGE|45Iz#dlH+&kwv#1aqQZo|78lR9ei@*@ILz*DND%jd z>gKWg)UL_)^_GcYB2k0x0$slGZ4Em`xuh`AeBe%n0@g!a({)*&U=OQtiv_z1XDi%C zE4}j6%z^iv(4xlp?Bl~30&YL!A*RlOvkY`yJv~DFMhVC(K?QoT0iP)+c5-s)2y7PW z`Yvam&zr3HFRY8G$`0X^dFP1_)m9z{KNQE06~VXU`s-<%W$ao^1L{`Oc6b;1KDF&; zsoTQ4xtb5OB~Q~8wXhGkFB*J-Pp^RJ34b1(8Rm^BzB>Ty`i*90@^76g)Sty|JwOE#2(bi&9L+*Mquxz`lF)5The+ofm*IhU$802gKKg)2aPKW9= zF>2+o<2cj4?Xu9Pn89zxc$fU-NpAr+@2aQDrp=_Y##3pP3Z0p(1cl)aE59z6XvD_7 z{1p^g!=_8@%S0^b2B#z1XDsHv*LQOLmwv0sNnXn*Jqx=FHA+rzJRhlexm zsHmtu%4&g~&oRR6XmOrAdFvI=_vhOYENeeaML(RaeK7xgOF=+DZ1*IUmG5Pl)4G5T zG$LUC7qG_5Mv~&*zN&K2O>jr~P!R~D1@mBACWb;kZvlm&|LEi5Yy9dB94C<{SGRZk z>waCE@r$~YAbMF*A5qKpZ}(U-3ZU*ars8|r&sh4Y{OGtUD%gLm{zbhMXFjZd-2fzr zasTl{ypzPd9AVYqv_Tac%Aq_Y|16W%U3@Lm;$nN*J5skqF&><`?CWr*yu0|&My)dY zPWocFAA!;IJDzkE1A+S0v%8kW_jZjfF%U1_DOp7Dt!u<2+aOkt_w_$g0H!*}b za6)o=ps~p|BP6?xBU_%7?iMdJIjo_JN3EFV`=e3sG*Z>MNcfX08rsr3xI^RjJjktk zt2aTpQ3zVjeQz3O9sn6#Qj-$JP`I<6hA(|0F)By~9qqZIwiaN;45#1uPT0alOx{9e z1TTCHErn%ctj7ia0dM${6Z4B`YvIVW{KuOk*NgjrH@AK+@MnbW=OuE4$2xp}Dc2*; zB4kC+;N)><2}DS!wEws}Y@HpSMoE=Q{WeAoB@;ahzxjycM7GI?Y6rwBk&IF=XR=SeUq`Sl39M+T^cZAnW}1_7k$`KHMIc z+CU*i12*Nq_Q2+N^+@d?;?XJwS=}}>TDm$6YY-b zooG6ZVqjj%c?5*h|2o-0gsdnQ-xEkKwJ_3K^!fSSh+N7az~$NbhVM$azE0*U1>WUw zp#zK8hL??%86qMl(`k}a^zN7mRRu#Z1}Q-jIZ%LTpbAtY&Nae|WbcdGx>@GP{dhg0 zA9K$ z_OH9UL0wYp{%>Yu#{S@D*_w49dpq@On`t^jEe{N7+;53dnDhLrjiw?u>beoN#A6F6 zzZocGK3u;!a=F_LT1O`X(S8>y6L5zmr?iejhLtSNrh6 z(s;eKzzvE&;c@f6oEoo8bGFZu2SK9_lfeZ1XxJwqx1blOn*4?XYP8&?K>=l{e?sde z!wN{5`3mwRE?JrG=V!84naYK@36E{Z^X=Xjn%(qCn}KIQTNqeg(SV=h`Q;@<@4xoO zn!_hPwQ+~sPI=sFKdP>-0e7BOGpu`|ijLF^UOUVgPqY7af6_tkasRlcDG>=s#bY)7 zBN_&sBW_6ScuC!rv*H8B*<;!Hn;Qx(75aX)u`H)!7F0_kMo-x@U=`Y>ii!m0os$La zM>m@B3p+8Sn@&NQtL8DC#D;@Vh_9t(&w{l4W`4Ai6i+)?Iz%%U`$ZVXc3|-xtpf|2K->Z)&(psjZz9l30Zc&+JeXES?g4D*v}@JXk2=o{C2H8{C@Nn1O6 zga7=-=5vzRNW3w~qfEEa(-3b^m&F62e}SKJUddi^2JA#L{9@myDv8G>S;~OXL&BlEZV=HH{#TdU>@oab zO=`0h-zq*np}$YD7*Vf*g$c%UIFyS7Z~fz$LGjaNed{-`tWHuh$#=b3k(R6~P+X$W z^ro(gHBUv4m-g<9`Q*QmeR*F=eroTk+tS*VDK%o+Sj^z+BAC_mEF4*DoN;FmpAL$x z98QxZ9(QHR1_&P<(W2_VRR-1{8$Jk;?eYuv$7)W<18=<^uHV((=$^;hCr8gCREF`6jdZbwyCfFpHk~G1kFM9rjeidstSf_i5{S3ryoSv&NRT(Y893&~D+sx|UdfBKCQWtF*>sg}$o03l1%wX!>6PCMFK1Z^T zc9X#NR-ENTT*J1aG~i6ngYk`x2bB$!^5p2=reoJTH|kHiO{%agqmu4`m|1n1t+}Vs zWkMT8QCYwLfu6)R?7GcUxlYbVJ-9_j_e8QtNg<-hHlG;5!Yvu-@)`w$O{&7Dt>z1B zmWzNNNv-edyGYC&a#IuLv|go!#eO4tLw9Ou4*B*2 zOl#u%Pk&lol(?Vhs!~5=`%p)-6zL>PkeI48(M}u8ljoWZQ_Ej-HT8?X(}?Djhp>#O zw*<5BB8hdvB!L+E*&=-w#N>P5=d$DF1(UU8;LLPPxNbjBX7C%?bsxWP+u*@w)bFK` ztJhmj_+P!fWl)@L*DaW!!Rg?x!JQz%rEv=q+#wL$-625ZA;A(XNaODA?(Xg#+#ROL z^M3Dn&Y78-e{&Z_QPth_E!W=H+G}k|OR6&?v`&vJ%p-l_ut+!q6b#5ubN6NIe~IC! z@gTPMx5FT)_*=jC#!Yy(B29_Ru1OIG-Hi}V8aM<8^A_$Kp|C!K?RgSmEb@~YXz}w^ zVc6{Isw&f7Gf55ym128`9^rY&VAa)q*j$5PgXO$wCN|{P*O|n?-d)yjg4&Z`pHzKE zFEWSA)yV0*wPsQ&96?HD^QpljE-Eq_7x&a3IkIo{z|W04ZyY){`Rzy)vnpVPmb*TK zsgy{#X@N?j>Yqm?%=nppC`$O1NzC}3OBO+H;ILr8qfF093I}pC(9Q+I#r64$IPwqJ zLm}h&xLj!C_@pX<@G`A!vQw&a#}yrUn=2am+c}oubqcJfbm=thoqXFvw_m^_XJ(W% zRJ|DHam4u6V4HbZt_K3HX@s%)SO7Yp1CXr_Nt+dlG9qsb)87EU`UQ#SqOT{PylG-- zc)PFFcG&Eqd3mIBbN{Q=Brw>_z@}rUTJqei>lk8u&(RdLv{k;evEkh?ZO`laJ-MLI zq{*!*R2bY$8*}RF7RNBWbW(tSOmfZh0O4nc4u?Wr^ayZwIYo76NkcsToHed06qr3p z?(U2A=D1tP-6bbywQ=RQ#fjENk%zSlYjQ`%dkb-DD?m1Zm;R zTix5ht#hnI7QOx#ek0!+s|IUj7_)m}1_~PJB3|juV@K+;`GzBVZ2-TehjsZt>`68| zUOli2JOpiysIP-Q7l^3a0DoG^EWkww~cmFCoaz7)jz3ISYa6jpPulzTx|{AMLwzo3Qm&`xxn=1)}U z=(%Pu%*Fyc;61FOPnycWG(nq@Exs`~b8jStWf#J2hmR|riXFpnOjhXL+tPmH#B3T$ zIp|oMenvV;&V1cz<~M20aXt6=Z4h|Yt?q2{b6HAsUHjmDwvhE0aPP;+im;xrh_=J7 z$GPLZkX@a6)-OFZZe<(N7&mY>-803R`wcmb9$@_=IZQy!TR;416oSTtcE1@8;<|$O zZ!O9fM6a{p)+=6@oJ1{#z|PBR^$m9g)WP-*eHswEdTkwX9=!hwy!4gyb7UP06!~O< zIX1EdUvS!eJV`_nFADGc%Id6A>W}CyPKG=$%}%2wXQs!q)e~l<&b!WA|9Y;JSm;oI zawTwwJat<9Rbpc!5qXK^4j2w%K4G+OSj-e<2A9ZQGl!KJ9yghPg8zv?1)2q7!b|7|Gm1eLj<}Oxr;%*byisE& zaGk|g8waybo&B607uEY%2DajZOwQM)njqP0RJ8Aip%5{M{l<5YVk# zzcpaDPTu;727v)!Q^*OZUcwVC|5d(&PWsw@j#SrW^}(bNfV+#@igfLmB-yEJ z^G@AW4o`lKZyv>iNQ3H3Yr_r`bkdZ9&bkc$mI8Cg_JjBvVFiI=;pEC+@&shbVlLW;zuR` zC0sq4K9ANhH%#K?Q0Q4!RjTjw+3b~vRkfMGuOkOhyuLBDMN063CqK}nIlEVqnrmQ_ zKTeMsGBJ8{q8)yqkw9hG*Bl_B5JqsC(>=sx!T)kIZM%H7w79N32O8Pgl1M_vct{GO zwo7uLW9Oe2N#>tZD36a@)?P6Q-JrHOj@iP^x^=Q)C9mx>kQ{fv>Ow^oJF(u@RM)?` zU>q&DS6uywTKWygzI3IzDb!j12d`=Ko=R3kd4Ll+w~t}eoE!L+p1@K1j5v7KUc(8| zoE7ri%;pA0^3o;o~_4igY1iM?4QTUyHb1Ik~!rle({O zKNQV(5|onn=zN{qdV}@Ay-c+1ql6Lp>)EX-MEW89`n~+Q4;k&o-xy2&-iZnp-p8l^KF+l1-Fk7PK7Xqn(-x4rUzRw;5X( zLJ3YKW#_|BKwgR3i-CdXc*@zgui<^E0w;$1B~b{BFP13zfCCk)52%MW_LS z1%WWbfp|SV%)+~$B_!Z0Dk>IiX4V|?YVM)kSSm5k@pc=AjpL-6N{C0r;C-{xGBBPIvcfUf23jWYA$Cu$J}~B$D~O z{op|1zglV|r|e77kgP_gr2n zTQ>L#YOO56($C&FnOD1+YWcC-y&J?K9m+@@qdrZzmF}Yx(^nY-$6PM0%!I3}$v77-^k9Fv&{QXt%`d{=f`H zpGmgPQR}{S3T7JdpDqv?ZdiN#ZhtVw>@k~J=ZJczQV`=PI$uCF0kET>-+J#^a?x8T z8h~%*^7d5btmK+ZLtn?6<{;IEO|po+#e`Hc>2(4Yaexgi*K2QCc@YB8+{zyc`k0xBuE?8QLdbVXX+FAji+SvE3Iw7v+?2gwLOO z^_G@i`h#VVC|ANGcc6ptuy4ZE4qZcLksM@+>}nK*CKowR*AR)9DZNUZAe(3}cb`T# ztN~+mFI0w|RmL~}JJ*-AWRrG_l)0<^GX+Q!GtfjTAY9<{>V5S6D;k!vV=q$r2|2#U zkK9Fq1%ehn^j{1(uz3=sDCrIIv-nEDo&~r zjKZH!_I^8WYM3K}%+yJvDza!DmOCZ%w=-F^`s?>>r*-J3XOKd8WDBA-?2GaBhAiRl zcvv3Na!7E+Iv8watl!jXi-r0>J7E8LYLu3dGeKS~OGSRz^xA~R0B zSPd0gnM>?>NzjWK!&Oz%GRat>Lqd zYQ3`(#e_x(m8Hy^C$~O3Xd%tJM?dbJQ__L^MLUbGwnv9;<5> zQnhUB?iyx6c8B5jb3*xQ6R-|5&XffKl{uFF`)eoq9huC99%;JZAXf-bGe#WFHCD(@IH@15Db#Aqk7Qn90b6t*2@jcL_G zi5p3$o2Jb?wf7lBNOKS`(i(&NHvO>Me1{i&f(EUMYQ|h8n4h1gV_-1maiL_UVaX1Y zuj56u9u#%ALhLBMyrJw|B3^E;KCFw$EJoOX6ha%|J(2F!P7vAFNXjyE;X=`ELBLBj zU+fWdB2u2sI{rU(gJv^h_zrsDW|R7`UwHrF8qH?(E5{c;G=5M%H-Qp`&|_+dlz~b= zHl_-TfKENUBIP}GinNHMS(J8hMkT>%T=AQjzHdC;nn#@s9zXUnURERYur!0Zv+0i{ zH<*>HzGRG#ZlYK#GT||s=TUfL%R|@&%$297NYxX+1)V9V^y3WjtUombXOq;Ir5a|% zCChkFgxk_BW;;Rez>5&XRe8s`fxzbT#9VZLfq`E%)z=W!MfUu_Kc}^5*zq;?=Jwas z=k98LDt*w!s0M|lAB$`}p_VVxh!GvJLLA3{a@l9DndU1%%ZV_k&`}Smf>NEzd)*Q! zkKW!Mr{icd?zTSR%Xm7y6|Y&lMdv61wq$kntszpjgIh0j5CzH7#a?%_?gE+(xi(l6 ztp>7H|9f5`Js_fsy&UUliKwr8rl|CkM!=A6`ZK?6@A}K z<_eRVw*}qNmBhT8)GkflqwhAhnQsfY+Yv_wext(NpKvDIr8=y|baG?rDpSmLP=rkA zJ8P45&cGQTEd=xlIxDMMY(&Z!G-NT+!QFqiG~bq4{O^5_;k0V7?7hF)nQt|zDOSml zfI0#F=pxWEuQ?>ToXV5VC0^y5;}k*=0=>jfK;*r}Rb>^A)OdU$U2|Mirug`a`vG<5 zI&97`W6Kb>O zE;G*4(QYTKO~Ek+H6JajB3#Y7E{(V*$&N;e+iN~MIMOt#%#Z)5J6ea41MBLjJHGZ@ zxRk`3wB6fa2dor6fEQtg(7g!M`@|=5h1<_elzA{ZB^licq%7`iflNa^Yg|qRF2S^V z|1%HlCl_>@IQn%+?P%!SyT|K{7#{8%>3I5dlRr{OeRHQZQ#1=alvX_ ziIfYtP;7gu&2h~C-lB}eQj97!^&Ru!xCaHFs$x5uwe4xbwg@HekFZVzsCpMHt9s$e zBzm@CZ+&hM;nX_S=d+qi`I)iC1`-x#mD7GD1ODzTrKL5mXVbFW;IeOPVF6k0=;&x< zi2uU^5KP;jEgh_#`KYh&dp!Y`lg)=LH+iT&6=Aj-Y;T^>`knp#S!waPKbkJoI9`Dk zpO;KZ#J92vhR8J@*=oiYVS`AO5CKCB?Dgv3eG#R}etj&Q{s!Zil$^}S47{L2Mt(o* z9TgR|oz9$}nF*t%r4_T}Qz|rDq8F5(zk^(pmX`ML13Gl=J8v!%1A|y7I>}hGmm9bS z0*Z8Z@@^F@n(_3Gt+$P~-5Pe$%}nShfk1--=c6Sp^fm%0JW0Y(# z-c4_b;c>(0Nm>13(2+k6MN|YJgQnLO;^rNTjlw`W9mv2FwgA^W!8MDXu0-toznrE@ z>Nk|;d=xhR0is9aANEKDApMlrg`WbF8cisc?%%B^Vb`HZIBerzfYr#h11mmBWmBz) zsq|S~=t5#Nf?(KZ{!nlH*wQ-|F8N8Mzs!zxHNuTgMdEf8?SXHEps8pNSCa}XsZ;t3 z48Io7asLY#Dm?$no_R@Ea@O}s%})S=fy&hu6!mVLR<{B+CU(mi(9J5Sz& z)xF*yZvGHnq8nek{$HfH2kV9X z7b)s(t6Qw5(G?$D*{&icP&TjhBM4re+mk@mjV}}<%qXH=ftbYn$ZA{k_bYBoh-?l4 zKC9=_vHFyd&^wQJzt_C-J35^4UKJJ zJFPs%YLqvIlL51#)AD|G!cK6D3EdeY`8Wg7XK_ zU>7Y}<0mebL7GWX?N&Q&X}Uc7ZnN8(S;kUE ztoDB6N7F7H;|^{>*^#Er*Dz6Al`kowH*I=s57Nl`{Pg_NBR&tWqo1vVR+Mn5W~bmw zhuwg9>2&Zk_G-H_xmU2Y7j z^w=wf`DmL6Ke0py!Ho1z=!4xtH`OxBy*Q57Ukw*)6wADGI6d!1zcuC#nsiFpbzcUn zupJvQ9Baf@-~jmV+ML2FFZg3iUNVfh_Ke9Dp)-${V-pe(pm~6|U(>Bdu&RX|Y-v}L zU;KV*s$K-*=0-m|2(alilIIA9Vmj=c4B|QE)2#=E-<}_;O#G_X^ z|BD#U+w8VS0qfSQ0H5jz#O%hu7!*3=$IA&a@$SR$BcB7RWPwUaafT~p=9N57GYqG< z29+O(EYPiol3Tsu5H(~KWc>AZq%C}}=i0$GihTn?uz_A@S}ogXnPH8Yj31xjlXjs1 z>}doKxy_!uyVteUW&C>Sfl|BbkJQ4)Xq9h*2cL#k&PWA9X(@#C3+~tnL(lcrDji9i zc+e$9`v4#p{f3zH*ee}SJwL9oX6__!?n8Zq;=ak;FOeZVe$4t%7E1<&6n2Q5TO53_ zp~3fPXIAp!e>}4Z=LWR7v^`Qjs@FLHC(l%~!1gGqoDMbYc7P>2NY>$lD;{?Vn#7kp zXPs7@abdN=)g}6_d03rt*Qgwy!yY~9yflx(tVehCs0Ss5S~-WqUO3`;`sLj=c<`Ah zp}Xy6|A*+|2_Np>9eNz#f`ShXencyK??j*B&68}@+$hW%{~}(CUxZjoF>4@IV#h;r z_Zy+fy7p-LqkAQ3Fn}~ zzn(lWx3G)OWYkdhq$iRH%>bk9aCNk*XT%br6uqdTWM&qO{1U*99 zoLHNqKiJ)LWck7Sg637ZggB`@X`}I=9bVK7ZiatF1C(&cEsov#KB{Xm2)JPOehR_A ztH@a{R-zyAy(alicH&uovMhOUXZMz63p+D2;KxBo+G5Z)h0xu>;|~#lMPazJwRQqh z>ARM)X*r2GU4x>zL?kRu6nYU1iM9(ch`uuT_${j`%i`AyNw_i_lNo3Fo0-ZW(o_hG(o?zpi&T@NS$lN_7w7qW|o?7+JWF@?^-yZqCf zj~nH>nBJ1~HAKO6C*l+h0{k%1&5a_ZSx`yvL&t_9MM*y_j-QBj4k2!kfxiL<>{^(u zyRD|5v+AxW_+e3R_=n=g@WI-g+@!6^Git*!y~HW&ud4+FX+Jz+Vd18e-<;zNXo9tf zjDk+Y{o7`2g<02(vFZ(&L2SGE_;xHdM7)N7?&Sbg?8u^w2~^G*jTiOqnDkmI$+zzE zF6iR&;;9M+7Erw4SjW@kBhBH&a1W0o*mC9?mCdw-UK&Bz zF;oo(yI0!2yu(j~fu#CB+9_TR2Dk`Bl zS{T6k>e^Vo_zjH!OzG@i0dw`JLGbFO3bJ-l#Nprxx?@l|-2;AvbK$zZnh-zZ{L`u6 zkp4F8=;x;-!Bu4fCEGB-AS^q-bV0f>;))Z(K=tE}jLSres`$?OG`U}7`8Nh;nvPwP z&#sM@i=II_?dD;XJLO^81W>|g!b2eqP)a9ycclP2IDp3+_7G8c^eI2|6FoU#OeOt8 z(B}D-F~zpv_uE~U+wvC`{Pj~>*^%|DAbs?c3;T&Br7wAU>s+%KBkD6{+=--{*3=1P z=fE~yT6~C1Zi;$Z5|>0&9w()aFG)dV+iwbV*+q9aU)-;!KJRO>g|E+Ne*q~lL;fa^|HC+lWxUJ>u}#k4q~^~UIl1aBLnDsy}i^gRql zdW?uI`^+pRK5IORvCL*X!8`ZJQR3R3S>~Cd8#gKT1 zam3mkDBZvYDkF~U-*Z6eyoL>tDaDr>;M7@cg44Pffn$XkZcpTdw3?o9v;g}8Mieay z4vy)B@t;*N4zgG!*)oeN_V>S8`p~L13UHOeioOv}^ye0yxlatUiYdm)60cQv;V9dA zb=YaqWhxeo`S`zemy=&IYp>;f(=v%i_2phPd$r1c(%y<5E*CnC#BCikfH@jPq9w(x zh5YsUxN}|y7azj+n}IXY&6x4(c?@2h)}jX#Q%NuRF!Fe_zLMlmqx|fos{0{u^Jfpf z`r$1A0`re3T*^HqIq`2%_*?!zqAsg| zkEWOO^e_~e))rt$kzFSMa_m56=;H!t^$Xe~yTWqc0jH9sKOyi@Lw5Vgo zvf!^Fuh!Y{e!TUd*)rP@S$b_c5Fx`@fnlhV!m5i%w;lS@N|TG{FG$noMv0AyoBUbk z+l^8Z->w|o*GuQA)rs`8FRx|llM7n)tf*-)DhoAba|xWH49#1m94doRn3^mBz{#l@ z6DeWPwpZ7gwdI!FwOn&%NNhapQgb5Zg{nbZkwFYT3nXxYtflkH#Dk6Gl0UMl?s=fE zT6HmAWDa(8I!Pd39+jX}0>g8Hf`RbY+eZqni=_fxf#6D&0|R%@LbLqJu|vmtUKf5b z9#u}wC6CNix0>6w6-eUJPo)vq4^6PM{ptb8OOX_f^5o9io zQLxBoA!!0U{lK9g8`qeNhw>v12jRB_f|3eBz0GQFI3I`}eF*Ma1)8I+tI*O8P5e8i zI(c>I9z~DxGWZzHtdaGdt8K(RWYQbQ>~DNbL72EWTXU=@yK zf%VVQ;6pKDRJ^1>lt}o0+62L?3q@sPp+0)RmY$u4?s~DQJ z^(iakif>P=b3Z-fgh3y}fL(FPg)gBue_X-TFH#rkQOKFD;M;tvrnsr_=T*hZ=M*^I zC+?~Z;Tp}UzJyG1Ya{QqR<=!iGFH$Z{}q;33xoMC;n~8RC511-7$8D6pJ)0#J~>5V zL|o-N@*n)(vyMlk(=fH6V0FnM3h>IIc#nW8sozgnE_pbA267Nt7t7o$z^@T zD@tR0)!vf2o=Rl@<@DF}lad4UD0;`WQ{%$Vte*Lo#3rJ3=M>A72W;9uQI|$`;(*mk zq1xy2$C@y0tx%{E+c;38-r#l3ayQ2)wf{B|ex?KtQS zBr;zv#fT2)MNQSXPoQ$rAsvtpF>(a)dr~3s8wkf7i95x5?BSkl0NtM*Fj1jw+HM8) z)=r|h-58s^?oP-`LGv5|JU&fIZ#>iQ>oboNVl!$%MdD1{G)c&hTKlYhdv_M1s5BRn zSw=kmRr$6{CzbgYtc!8jpq3QG!Doc^lOLSxhZJ{@FY2nw*m1H&S$uS(lM#m}>(3y- z=m}@KHXc+s&OU$9ij_^Odhi{BNV#Z|^Nh6#qvHLZV4uoN8GqTg^dd5jiaczfNfR-^ zS4;=l_x~e2(7c(g3#Y8qVD}pIIw4N`dgBh}qrmf2#WQq`$6*m*RSr*2DGKkX0Nz*l62khgOSyi#v=tHI% zzU53ybTKvB>GJS(DJz(_mI)4kB4&R;_c1p2z-!fktgQ?F%4d{H;*M5HI1bG!#!^Y! z0C^!1wL5A%R(hi=`#EsqGhSa%&Y3&K7Yp4i;Y zUO3ROC17*&>YCOQ^{=;wHW$8x%#~*e%)S9s3vM*iuVgV7b8&1yiLjHaTg=AnXQ7MJ zo*wLc4q?Ld=S?R_&8roPwho^6RK3ZGLQDo{L9>M(bJvODvL`8_*?Mhl<<+y=bilh_ z1C!BJA*qX~Rl_AV@SPCsX~*I=uaF>iW44c58Zjs~i{Ikc6BYbpyMf7m`p6~(Lq872 zL6!HZoOVR%S-xO3QqS*}F!#@}?sZRFemAd>RqBNRH*{2}ieTUCoy8Z$b>tHs27^kA z=$4=S>4rvW+l!!*7jIv|O<5p{M{7k@FbbASG&D8+s^9D>$hIg3WL8Sq+}_&yot+K$ z!arH+NV>Y#jHIe^+g8+Q`$l%mu4->yzvj9FZ!SIGqk62IpNehL=FEOP?3@KOpWfG! z#D~*3Q@w69(-V0+`l3cb`)uGGq3k)Q!#QdMH_@RwP#4Vq;6)wFPHe2p4Xtii+lfsn> zvV`VJK;N_CK=J}dp#L~_WHSuf#gGf3_b{yO^wRTq`c)125D>bMw5&~ z#i`q-61hYcS)0M}Zs^%mGymle^`O81P_^(w^>;%vGn(|1h*$m~?L^*Mtm9#*Hq|~d zQqmpd;9wNJ8wZ?9`Fnr=-{?z;l-00fxPK9t|9U1bpZO6NM{T|Ytt2a~SV!#%(~g0D zN!M+va~i|7rh~I{+`cyQ--kH&91U_#>l4Ei1o6(#&dw$V*#llmfh_gnkHt^Zu#Xi^ zNo_0|TB4yBA@mO$Xp|eG27N?Ru2nn1>?6=>*WsxDvW-=$R`46cv4H2>-Sl{7N=oP_ zrqJ0?M1jDlabkkgm1eI&rwmL0(*GDdFniNc1zIeL&q>qJ)-BPXkP^_H4y0t4varkE z=>x4cxX7h>M}wqaG+MqGg3|_Yh%{{>MZRnpln~9&)G!P=w%o!v6+ao#?CO!NO>I9R zQjTO>C4i1CR=M(ahn~stLUmyZV6?wK{}(|TXo0o4_35%yF5ag2Q~k`=;bQH0u?`P7 z0Yn;HXUzKk@-81kyeZM|b~};AfKS1+i|i@s7KHr3Z01vjYc;FB@)Y#^`0=$>uR(ie zM1@Y)Hq{Q9U)MyDIlF~DE4ajEo4QB>TY%l%ebHU0Sb|BFx*<}XMb9;TqH?a%f!4U9 zu_M2g{+kzZGSw9o3p*gJ+HN+^r(kgWAYI!mSk#BXKm=Sz+=!=)7)}19M%mJugd!ec zK3;fcqv1)=do5w~yzWj-;}dhCD}Di4L`mgMC$8o+?zwnbI~^Tj`S~(XHqS0Nd98FG z*Q7`b{RyuEn{mdvdjyUP!Q;kw%LO5~bOL~}lOKqBCX-L@oLc03~5_rnFiz6(R z&qd6^p*)48)%UUCK@;jf9{STC=0|)Tp(aoeun&q5S;AtsRes~y4^#z@NZplepDY?j z`PrfS2LoAY7HYJE;wPB?v`7w{oYxmJjxZSecbm{@W-g z$-E-@rQr7jjohihkikjziXN9JfiAoG=q<|M$(`8pZsTx}KXH0*_&wDUnvrY~R`ycX*N^_6bh!K+ zN)n?<_0FwT(UbQw5_9qpEaqOgkxoR4dwEoj!KHEmM|xd5Ns1Gt+oalQyc!IG2&cam z|5C=ycU;~JB1&p=9YA&eY}b=w5y3%7kLR>yMq?hjkN!uO1-Fuqj2>5 zhFw>0XmKF}*C;LOo-&2Ij)<3NxU@qmctC*XelSkOxas`Kt<_D@!`s}> z0uol#uz~abXKqZEbRfgEUU;5#J~4i9L2e;qNf*@;)M+46aJ0z8g^! z+Z@cFq4y8JKfD4U;TdD@2;gi=G+D3t=O435j7dT`ox2^svN$8!k^*HDrp$y$RZ}}L zokUw^#Vw-Qp^1Xi|L~C)1g=~gPaj|hodf_)thnW|qV^xiK>jZIDso_HAdR-k6JGKf zk25cfaaLKMR>2jzgthgnKO>^F#+S90?oTEd#^|FJt9{axu#ou{@r#1q0+KJIh~F=Hh@8 zS7coc65zqOpr)k@=Idk0Ov|e6D{1Ph)R?hhUwjA$N_%DMZ`GinBi+u^EIh-LLiV!4 zT)eNu)~i0~o4P}Bgc1-fU^n?JYrJ+#ZDy#sC?LcRbx{t>EB9Q)J_9(IW|LH$HAvAd% zG$L8%;uP%UN;y$5O8L-8a6)KESshXIEO2uL>I&dJmD{2+6URqgLG{m#mmVZ+hr~nd zg@_`i2At+ICfS|BVT+2HPU7leQ__BrN?A<62j^mJGNlDAmhRuet|+go$&y-!L>1v! z!xEGu)tPa_nA{1;8)#aPokr;@m(b75h_1lu-h%R8|DP;Xm<<%m$b~b!y7t1NknQfSS}43+0RIAWq|NEDfRqz>@-NA<7_@33gkdI~ATC zTt)wK(PAbSN{&Lrjp@k}&zUkO1Y?%4G2#DIH`HApvsnaRKqoZExBqsY{Uf83s`4wc zbG4A&@Z#~cE1GaxclYAx+PrzZBbZUjUSJZxQx|b>sv9Y@rmJfKJ#2Tf@Dlj_g7~XP zB@8CpEEY6pEwxJO?iM(w#dz*30QbLLhr{l5=6kudzS^Lo15xO25XT`rHCJ$ihzv`-Vc(Ty9;r!quSdeyf@huXnr}5T>h*h{Y>)4wY|@e_z(eCB$7?*UJD_yu(` zH;x1F+OsC9yo)gTnK)AIvu(uv(*LCXpN?_np!_4QzQBPO=R7-OC6J@o544xY=8$0e z^gX)I#fUA53y0;C9)%TWK=7Lhl=E)}Dp^$9;uSr=6H)DX`4AYY3I8yLy?JC%vG77| zx-+FRbRg`aWRaly%ESm$}*sp{1}i8=CNci3B-PHqWIzEh-q{ zEdD(a(R?1e*PXIAVe++N|g+478&CDWZs|UwTyihY0-TJL3$?`@BlSbc7S^kEgeG z)+o}YUED@nN%DsTyD>v|d{eRQ;1-IBYj{LMS;`Ns`2*Z_1rP>;R*o@n*6s;fc zd>S(rsZvUbP|y5(*I4@I$U?@rk^!kUk+G~1TwUAhsJ`Q{(Rv(Zv`p9;&tW~IJyh)d zQC1+V>F)nz*!TbZ2q#~VGD-O=<8*nAs0Jxwi78HyjPx&D<%^4tU;b26J+a*A<^Y|N zm^W=1mypo0r0bkdohkL{<45x6_uSkGel^+Q;mD?+KXZAkF>!1tkVXC|k(QV5`bHuC zyY$)rbRIZmIWRc*5`{oV7SnV6&AZrJs;^XmW`BDA1^XtL(5%3y_J7@1Z|L6u$opsj zZRXBjGWq3{pn=N&T9GjX4h2fk0ARab5|noc_y;@xA1_}YEpN*pcW=f}BD z{$%gTEMK!`?{!bYloTXU5D5^WprBBsrNmUApr93@px#};e}IfQpK0AfzVMyJwVYM$ zfX;44jsU2yrgp{vQfV6_Gk^-f$kg3o0Kg9g1ruzcrsb?9FUM% zWH4c2XC~$1VPxTCX6Im`CuL=3;bLOuU}ENEU}oWE=H_K(A^o2ZIYgSHDUer1OyYlJ zLGA>|&7Ga?d6}5p+}s%5*cj~`&6rqtczBqYS(#W_86YzloZM}ljX(^xP89#;AO>(U zakQ{^wy?7${maqF*v`dSfE=Rfe@w8kmzV!v#0i^gZF~oQOZ$W=2LbTykbhLm3#mHLB&cwwA zVCyU`CO{7Ph0)Z)l$VW}hr^hK6Tra1#SLI!XEQQpFy`UnVc-U^0fFq?KvParqyO^x zztMAjm0)A#;S^(LmJnlM5$ERN=J+bYDj~ut!NtujAu91-T4`G+XCqq^z<*_1KxF?9 zt?2)gmRHmfVB~D)sAgwp{ogE5GPiTKb27KHClwX_7j;rS%EwNP9sh> zR&pkYicEi#=l@8i{|-Si^zY^WE)B@d|1K_oEo4PGLY593ZK{2kj-`(C~IB9Hr}q3!PuV#=i{=-f+~85fQ;vdGko2^F6&twDRxY z9eln{7xt)p+e>S2Keo9&y^Tkr+<3U;VQk{!gM2)%mdvu;dsS3aYFB`D{ziDgMtEP| z(X$BlJuSuma#NSKDlx@3enIh+;%-pldJp|ht3u;JVSKGdX^QGGlQYWWQ=NO7G%k~T z-X8E)uT-US@KbT=j!ePmCQ)6>p!!S;^HMLZD(*m=M9rcp>ky!kF@B~JZAKug|7TCB z@9Rm4Zl}N@2eVV>O#NDxsQ>LJOliZf0B!p7rx)})U4}2_4y`LoJz@3Z2JHDrw$5M6 zCV%Da=uQ!!b}=J&O|wTn?rG(9%ROp&cT9BDtd78IKFa0M;vo}X-<5__m1)peu0=KW zsHRHbD*Wi(M^Ma`tmi+_|Is}bz&INqo`991Rj?24g!i^N0R9b+AW*X7&OaGG>Y_p2msY9mGkbJUjdDJ>J>M5T~inPI{PhOZjKzejf>LCr? z!K7mnQ?|Wr zTGL$tp%1I;*rZc{PA?z`` zjc<*a-}P}hHi%0QY>HV=sAk)>RDC_;7kn&bS_QFO;vyfz0g0hEG6lYs(XV{7$6aC- zVjQ!VY70mi))Q(;WUXR8fyO)=Jt`+*y#1Ot&r#2v!=I*(c{B=tYc?{{E@`VZzS4QN zV_M`7@B+zSeFjS31`W={2GtjyDp<)IXis?4?sUoTKrX9xNiQcQr%w!NC%g-HAo=FC zcm{*bq)uM1sIqj!T6(b=@p+@=NOvVWpE86l2ahTHdPjVoiJl*`B~{-+Q@mS5Q`SF? zlNdAz*Z`cgvgJ;)5|{ zC@a>i=CYfmjcF!dPni~VK1f+Ch}Lw}e-g4J;WDe~l9;(G)u|WS2j4w45Y}E zd@?v%GMychj+%3(W>CZqFg>KJt)0z2lpBN82xj`mcslch1Gl8m)Nq_O)^uUkS~Vh5 z7VLULnq{q{C3TJWX_>ugp-vvP-3H}XmB%az58Z%Sbd9&|*mC&OfpVm}=zJ;qD;N9Nz@Vy{SAHeE6b4)K@OoK3%kybqGnAg$a`Kr-~MzGg>;8 zya#TAn@pg0B;A!K-k2os>c?EcK;e3+bq|LEMO-=?mv7wGDZ|{fA>IN{_=r$?~0Rdc6gR(PU(0H}0M`?jG`Y&(Q8iXp~y0)#B3c zv{I!Lj-7#5tlvLRgX10KDkBbQ=5kBl(se2q^>9N<^{-xB@0Yh0U+IWB%NTqJTIFvm zt{INC|(%l>{%i#3$0H>crKvFnG59Dea(% zHeob!s%3Sv@a2>2FeSM&s_QZp&yqq0=+9Tye3TvK!lb06+-5M|4dHakl9b#`?*~2X z3H7v>gqi*X7ycqwVWC*yVldWMRKw{pj}3{nltunv_a7rea=dj*n>+e5-gK?mjUCfx zr8+*H4Di__OvE)?hQ1++&SH6M=ZnuAc^zOum6uUO7j&Da-D$+@F2a$_`-ZNx(O=%}1o6r%&uqKp|iU$BnMl$c0`vmCkyv zb`$VwFOrK)JYj~0nA8n!m5Qj^0aO`0!nQ3M+mhPd;~l0xds36(?^F)Dr{RB*mPiBJ zEE2i$P*$QQXwH^exOzCM2a67G16+fk>qK3Eck;U|Bm6mdJJ>enGq0b|N`@_QqnPae#r68l{@#);I zv&7mQ`a|XlDO2kd`a|_GvtU-2@&)%}J!(joQtW)Ut4-UNL0P)v6SBRV-Y9~6X;;j` zGlUxxptxH7ddfsbx5-Z2Fvs=rz@c6wJ>E6NnW$sM=NZOQF>Pru`>-W-kP`W)d0!}4 zSlE84G2yOY6!|Ewkf4;l?=WzvIRyn`#G1Nh?8q{IN*%U((OK9rqOQxa>#nh4Q&P-8 z{6)J7$;rVkN3$DA7ns#fbBqOr#SVqcatyyMsI|yp!)KYKm<*0lHSf=`T7g=!DOIu* z0oDqd-EK6z4uGlSGHh2J^x|+%4Ek#*vs_;dvhfCaWOn{H1@l{SjtV*YY*2j{kW(I}%_Cj#eArd>Ib`gAL{VMmk zE$OEN6pHs)CkZ>{F)M}cAde&5)O~}`O>3Kn!67=R@oPKomt6s0Nyh#VTSj_Fx~Y2| zFEuHKqj|%Bpa~E&VM{lXEd60X=@+sFU@boTNoqPy}>|v>8&zrivi6`vM?1AWuEbB5BVdAtBOwx&4u*B5!rz#G%*o%j9znj8Rw73D6dL zCKckcn}B|aJ_;H6;Z!a5%vicNStP#{`a->8QJkeya+U85he8}!8?SOaG}7a!>}%Pf zp6j$Xs@3XRG2E$HWw7B?YLWO5fK9K+ff6%E6{I&+B;V6uM79?9(d?N-U;8HX;>UST$FXD*_*(%w*x$l-wqwO(Qv=4RP^I92?`5Tl1j*v!imJe^V&@J~xmn!+x zVN4-06EXfnqfS!=BQaHa>}_x$t7;uJEsX@Q-9v@DLi-_Rc~(4>Qh<^(n&))BIwv<; ze{DjQs_IpqN&UXwdWCfpP+ygaGxNN~u#N1X;sXM>22zF1pJi$rRX!$xX zUZ8hYq=tI52B=+02{wc_2V95tILN@F!}8G?*nd|q^oqEE{R7dxz4h5{8yYDYI>itV z#|TiOP~ZQ9sMaO?$GC23+owU4G@V*x|9IJrA_omn0kcZTVPwvkLC_{w{SapDBDaju1vE-h@uT^d$j;qKsmf*LC1eix&x+9G%b~x^^g+PcY~Y zU;rzI`5<$y{s+pHI#0ClASso*VitfSZBZ_t%|oPEDwcWUzA=}OsP)}krH#u4_hAET zi{!#u32?G#1YHXEZlgD2#8Ilyv#C<6&Qg8*h3HUBfqO#?xz#jvxzTA)fuHTvV)^dZ zi}|XaL+#6H>lw58IBDUOQ!C5s&PBEPW4J5P;#RLR*TC6n>)vXpcd;+0K7147{z9D@ zHxt1q)hOSc9*u&6;En0B!_vv-@9NK-m1}+%TYXzQ2wM)17R5tetKMd`i;XrRJ}*xQ zMp~2=`UwTXej=dj5Qi80+-&hk+Y> zRJxT%*^};5#^`QgbuATB3dz3nn$D)b2}~yEyL$k+#nd| zH;KCTQ&c0{@!fVbUJKn7b>BGZTAW`kPo&vpoS#a_*)n-1ue0O_D3|?Fs!PgjDs?cs z&8*(25@bUl3`e{y%*uJx@Xq>kqUWW7f>|S>mWM!nQrt&dNxm=iO`rjsu&sq9ww$nxB^Z>N+${4r0lrgg6mS-al{;T& z(&i9wj{eCK@Z2%;Cir2yLdq6>N11q?i#-?=$5a$>cSYZ-BVO)ORxdRA>|?$<)2U8U zd`rYD_1kfe$i%*8znriB$gf-?3b|^<9XErD?=Ix+sv3)K51D{^E!%rnTP}+mfrk(h zWiFd}YCd-is;@|Ym@xFJ*`_Yu7ymXk`Kq>p#K+(zNCekWG=^5X%cU=p%B=;D_66FF7;b>WwmY5u-w;=S0e9pS=zVu`jz4PQeWX?<*ZWEK_0 zKfph5WQ7~~Q+Q_{OHGCpd-%NoF_CDg$z&v*1J8bNV1K5VtI>7^=P;6f^0J^qs@psh zc;NEw%_k8jWw1@?$vRz56guE?Qm4Ze{&)TH8KBliK3foF&F87&_+x`89lpiEWhyrB zUj4K^{{CI?sNqUg6ZKhuJ5pj{s2Dz&zWycKSEIkar$_u!{3cIlqCdV;fXPD|7lwcq zqYu|t{d`JgDkvmTAiD6%?8`AAP+*;gfIPNHp@X zNg@n@J)%-+u&32!w|F*;`l22Il9^t%W7jKo0Ty-&IC``*l>(?6sL^yiUHv`pcf#xL zclCC}GaLk6uMcRZz?s2angIdBuGE&?%F?RUa_$pRUm8zFc?}ThIbKe`(L@X6A=yg1 zx9PD51!CF|j^A=0*YBx+^eXj@d*zoFZm?MT(8bvCe(3F$Fx%%bZFv{D#JK`bnUAYo znCwC<&*J3~rQbH5A@fMh7aaNm6Zfj{Ri3BbLfsr3odqA(?&%aqu{xowqt{MEOSe_Q+4x`G!@DD!SL;%BIV4FO!yJ6*uYf$|#AEIP*5+|7gOJlNIr86w?A&hC{pv>26!vpk!|?9FCi;P>jXcWwF4r`xa}6BOAVP2rC5 z-}zqrBVm#2KS?duFiPVj6p&r#WIyI zIc9J0KCCDsLw>%&vuU|fr^Q@L{zvq4Ds_b`;R7tu+8OBizKj0(X3lVfHP#R-6VqFA zGvbm$DzIcv5|GPUx|hFOlDtB#Bluu3!BdPtHVsy$;7OF&$@Ll=APSAo!$@m#^(_@8 zxGcCTGARZW3|xX=buCv^A84-O3iCH)2fM%u2|+E;;^g2CMUT#6b7&co#vEQ!#d7eSFUGC2pJ~8XimbYDKoPW0!`iMJCdrA3*0^xC=ysq{b8vCM zOjz*zF((hCH~3!3)inW3Jw@Dl zb!OH#o8)$TRX;|3kDIeUc_Qv6lM(NueMa~QDk;jM3F@So;;AjUP|_yc4n0g3A5}vR zr}06HZ8QQe^Re!_3}^zsl^@Iz!L-|Ibnz`XtCg9prs*CxhXYO&L)o-G)T62i$H$0Y zc}r}~4H2`2E!4;?rj4#VN8G7$bq$>foX_M_nZG_d#*)q+Fgg^_S_V zZ>4s47gDn!GUTiXe|DP9cPd1pqWS zlz@N$)4|KYNfk8*Pjd?#j|nN&m*K}e!x~BM$b`_(6We!>+k2-1p8X$%rmHNDg!z`% zq^QMphgfu;I4BxRrIf;{Cf0TEv~wbXKyjkVDQ-*LPyLb(+BE6xwv>DT3@#Q$N;QkG z3A=5j1yr)pt2DYlq8Vvj)NGa-DyrMUU2iijzfMVfq*89+Fhfe`v=) zq)sv4`83M6Y4Hqa$~DBPcj|AHnr@!tECx@-ig;9@t*#bY+Q7HCh7`Vk(H&2`C1n%?q|O$j0=ItXvv+=bZG|M z2e4m!1?#gZvNb9uUYTj_zpVC}!kTGqgOh>OZe7fXwov^;)^<#d}WVk zcX=hgr$=e#ANSoC$T`DuRBlZ|$=XMR#>&@M+aokFiqfSwQpiC5I2qfwJ&li_@3H1!|a zN5N?7&UZC;kc=Be=d!~fto20kVg3(fD1!jN8S8OQKZkjWRJeimlv1mLLIzdX~9#KN|@jhBqd*b z&Gqh&Uu(K!UN@HVFsXons#;)ox^RjHJ@ln%eg+w(k<0HXd?8jhL9WyabijHa`9*0h zj|~%cq-paX24X!J>P6kE{E?q}=-#I%cDcX9zdjN+@|kAQg5cr7M(mrtg!1*~EtdMi z@|O%M7Z$8w((bm2Al$G|*^Jcr6H+0KoyFeu+ zvEHWBygV)nUMr$#N^q4P$0h>RSZBK<2q;xolK06A1e-K@$#Hi{e)+wFxsUz^5t!Nq zUtzDDQoxfKajyZBTn$o@H!WDqr^a3)Js2hm-$jPde-`!+j4o1$C?5P!U|fdz7e-gO;b5C8oIS#_^FVfL^9TYe&=2Z6&8%t=dwGWX!kKNS6!Q7SKE)-@gi zwfDX&M?5#$63br_r};1{HntNCbZKkve`-ObQF%uegOJ|!Q5X{u|I;e9$-Uy_^r6l_iD@jKr)m|woT7j|3daXLJdJZ%|=qx-H6Mv%1|fEGRn`<)6ank zs?G-n*oHz+fs{~aj9sCH=ygTG?5s3hF7^C;-i0>b;mx;LZ}DAKfd{svWDvRi;cCz1 z_IROMi{5;>nMJ43I;^VWQKpEH+nzL$xGsf8@DV0>6I$u#sk#6}Ke49?RUa9WQHtJt4WZ z;g^9oZ|U}`jEz;^e?Gnd;X9Y-NlV%Fp4*P55Cs zI(ST7;-O0$k(ZR~&zDGWsJ`M5;#P0jeH%c`k)Hv(0L-v<= zJ&+*h@yzEcGnb37LH~(kbYVe(sV6g@_19UniXG{fa$8k~Pn5~ef}L2Zf>mh5RaEp@ z{Nzz<#oQWpYmF$POl{S`teXNiKV1o{KxG#DW9?sVB+!(Mqn#M{7De-wF|-DV^8_%4 z-cZQA0-J=jAL4{2+uX0V_e^30aFB2rk$pSu)&wm8d2=yBuhg)p53~RPVBkzzX=i6= zV2p-_<{ZTDNFpQo1Ri(E8B5HHmWGzznUEyufzW?>BZl;v1)q-DENaN2|6|Q$3BOcvjW<%FEci z4$W>gE0-*Z4WR4?^S?b`#qi_UoK2S`$VmcB{#H#1s;teCMmETXlaIPIgBWr_)T%sD zH2}+hLqGnL!v5y-FHMn8`i=tCFK4m+wMoTv3$S{=aLk@x#*5UKyW$#7Gu>8y-(PIi zcs*S9=eOwH&vm|j4mmUr&JxFrP3T4oq6QrlrWC0D#KGWnTXm;mY&#cmKAu;$Jz8ny z8Wno;G6ez$y`=msu9ki}6+7z`+gQz($3a^_PzuoN!`{WBRh7CqT;C@rWMa_zyA03M zpS2ls8B|y|22ze4?|jM1hcv^5SkaGX(USv+MD)4d5fnKTQWXZQ%-6XvSx5cIRc^^V z`filkoT{_Ln7kt`v{b1v-cl%mY*F?HMUUeBRAP+ZYadjd;5!8(L_ivjVZr#I_J`$1 zKbno}Z&ykbd%NZf) zWTfQYXyWjYPh&vnehh5I{5nRooQfaJ!j!f7CEQ3}#cXM&6n?9V3Fj?Nv}rXRs2Mof zf8-aE{jSCB1F#Kd`Nm#~|KRIP<`f(s8!o4piMIL0C{ zI-D+1F<{PplH#mC5W|PvBG6&WzP*gu{MzaA7nk*)%Z481 zL%VHy^Ao~Hw474>MOf#z)CW_wjKYq6;#$qEu{P_{q!2N?aYJ?17aB)#@p=rKDzM|e z?DBx!>~(23Z%C9Ndm?Xtik)Ke3q5T)3qM#o3lwgqJ)x6n*;BpZZZ=uZ0ds+cA^x#* zT=7}b;S&|o=vB-+J9*eoC`o63)XkLh5(sl1F_nkEFqrg4{#F!vq0Ib=%b-JPwrP>O zA#ll;&hPFR2qrtiYB!f4hTS@zt1`TQ;&IyT_e-}_QoxR|l^1jr>^!P$9j`M}JQ!B+ z=T(2aJ)s-A8lcLS`Yq{DTAxcJ)gOn+mv8d?{#*RV?

7+#f&LstLh)uU6M z>LG>TICI*njs&?_Xm*tlj7KJ-7fsf)q#o+97cN**Vfj>vm!#Uhrs{C z5Y_`t6d{+wBXwMCF7tcHK3;k)uacWvn?$E+ywqs3qjT1FWg06#QAd3Sc-uaXJ<6>+E=$DTsEE_c=jUrjfj9~jn;G5y|AA>O*8w1kCU=DS@4<`+KeJDCdc9Lc651%A>w>z6Si`=F zGvU<=y_Z4XodBW+ZSaxyF%TmJ#?I3QPW8g<0Cedt-@H)hdwdG5+38;jKAJy>P;=NI zEd^Kk57}ambp9~E6WM8%0AfSpAWKZkT_htKZB}n(fousHuW|2$J0>e_?kPp}L zs>-(Q1N7Tc4TgBGang?Mujx(=V;5qZbCVlSHYxB&gqJwwOjqaSuxYTiv5y@lSC;1Ya`L(m$M*&l@4bB9oKex!r zZ{XtIP9mC*Ho~-)4(RpB2b*u|B<%-Y6pgjDW9{IBc_P7%vBUOodOre8Wu}JgHl^Pv zCmupaH#_eo9m z$4Td=IMvW2)U?=1rz~^Bvk2FlUbAz<1@ZcFLjNC6FeC!elSiGnt>KM3+~}`OW`AZ4 z;0uR>pm-;EQ*9{{)<=b^PKshgM z{vx~W9dAD78gcn?y;>a6%VS@;Mw>Nm_>K{WXo*OK-0?l)M82s#D8gexY|(fk`R$#85Xo;~F4@v_dH4udx^m#62@Hlu zg$Q5?Z9AOk4vX|8I*y#A#$|6!c!N{RZI|fN#kMB&Ij|)ITSV~FJf4P@hw_lj)z`o2@E?DgXq+il2@_(ucd6LEn;X7*LONx za0)!X@82yRC3V?ce=pYwi}Z?@N-WoHI?c)CO$vH0qUJ6p*z`-~w9)CN@34nK2%|^L z75>eGk0QSWrJaXjy@x*K5o|*X~6(ulJbs8aMIz8`$VO<)3wmW`)SMruiqy2z16$sQ~93{-3}u!vV9|% z_UQlQJvg0DqKthz+}C7zO_rSy_-=;~%u@~lfk@M)CqjJlw~&`UW4u1+)LAkV)ZJl_ zMVTPNaGf)qqYzW^YHv&kT&&n3hZQZHX(Kro4s9KuGg;?yq(Sq3fVHr) zsxR==|Jt>h-2%3a|J|w))t?D=)6<6oll784bpD$!S_EeAel$T_asFB>$!BGk7=kKn z@TXdO7vi-K4qOR1Wg-B|;0CyjaB}xogfYex)3XZ6r0C|9a*~B!iW3mh2RC`8*FXQ# z4al>S5T1bs@QP|QOjff!*6;fn8l?LfD!wr(Sml1=v0GibzsZK(_q@@!AGgost|IC& zz=|u&27S3Wp;LNqBK(}P0Hu(Ena!~up;*PO>*l97k@a1fguXkvA8d$*)Y$lIJ2zz$ z9LinrbTM#+wk19HjlSjP3-ZrR+KkyP#*BwWxnZF@FXyv?ud?{N)fkG?EWH>?Iw20~ z!aanISC7vecjc`vsF}PA+ioz~@Nw4bvX!Ad9j(&#fOzlXs6Ft8!PW)&R#9FOC5Ue@o!i=lb-_VFu7u(_32*82v zU`VY%4>d9?=>tW1SH5ByUGiJ84x`i)x8e)Fso{w{3+E#q2)95u(NEJ|E?j!y3uRfq z+fz_`KQ~269LZmu|1Lf=0lY%BaTPRuSCfGgepC z27_-=c9;lxng;ITh`ix9V!80sJrUB=6MGSiEmEg!fWgF2djza^nL!EfB=s2sIeGLu zq1q@G)zZuf=M;mVqBraZUh5goFS&kADZ<}8M}OqZt5~_9UFJeoD(9(aeFtM5o{jm{ z3L(oIXPkc9Fp)8EJWle@=J;}Ld(8XA>4ZxL(C?^fIh=rvE)qY(rie@U-|S_=^=2{V z6(WfOPvCpqHlOED`|cUk#UMN_0<4FyEkHf=BN0P)VsEAMApAN?++8!5|6C7@m<7!G z=n#6pAV>q-swx+BG@_Au-47bhekH@<`vhLQ&?;bjLpt-~CnA#2_osj}2noBx*^DB= z-8`}xVLtBFo$wsRs88-yCVb=<#9_KM58f+!3#56wHySz&L*L5?25R6wULQtP*s#Cq z<UXxKr{1Y0;!^6g%O^6q_~=q)9d+VrGAz%Jk(RRSJ(G!=*>M<6*Li%?v*xWWU<@ z*WC!q5jycH3>a5YK?)~vmaL3SV?=a2tTt}_EY(e2i+mbIUBervMQXG4kPZh)I$Gu}6I}_+8HarrWYZ2y(J?1LUZY7~ z7uEL;I=tcrVr08t6f&rX?B8DQLZ*)AYeZ2o<9n#{oG>27?U#F_2pw)0%w&~ukz_=b z2WS+qF^M>|#Olp;1;u^3;)cI}=x$h2g;vy{cpsRhn8pX&y#0AS=m6eefrsBu5>N+? z@W9|M>LxYElK1o9W4ZDB1 ztIium3_rP{1nO4(?n=X)0pWrL0(vPT6}ga5=N}(a9u?g88gq4YjEvDzqUe2g%^Sgp zaedMsRkNrvo#P{&6_B6^zji3se`C?k)#C2yLzl!`NMOh)ur#DtJo{{q+JQxrUy1SO z05|~*A+-lC!7I{A2K((hrn>aJ{BXdRy=<@r`Ac7aiD-QU<275ft&0Ej0+*DmX46_4vxhrDFt9-tNK zJn1yqG7z=g)MQYP7xJpcbYmqM-5g%Rs#zT15@_hRfmmEmm$g@E5YgQ>!>9%uPT>hg z^VzY)GXhf&k9{CL@FDQlBUGjw${&iwrR+NY9>Ty_TG+);pWnaIz&DokP3_?a^H6?5 zO{P-qa+TPW7P2l5ry^*iID)k2cPkkiuwnRI-&HBXrgNxg3=mFpwe!iJ8A47U z*Tct!=I*HTx*TG$JSV55P&G}o(zYbiYX>yiEL{@yr*{6%z4s2$&@+YfAurUTc4Vp5 z{L#_tPTF1<*{sciBb{@c;MRXizSF^Ee78Y)Bp+HpmF=a18YNw5S+fr5L~|K{Hem6s z!phaSRzW6O-00iGnP~%Mt!!am0|t6$enG${CFVkBW7q-YGB*9#VFH0A#tofXTG={E z_y^>PV_p|`Y>bj^ko-`+9%v(If3||K19D&|)VuQMZu)m(hm3!S$on4EwnCnAGOKvy zSUG}d^0EYiy@4|{`wGh(E$Mv8nCN+l!-%8CZm7)sZ2XR<5a$ zNEJPNd_=kBFSu9astQQT*U41Vg5>Bwu&zfnPveWo8M6Un(Y(=rtb;&w(+jQ>z?G6* z#QM4u`j-0An=l5&f35kyFdL4vGQE-|I6#Z-H>7|;xc-Ip7_v%Pt&8v6{{t(QTy4@A z#L7`j8}Z%#XK1$?QxV9ZD1qMlACM81?*i-fW9=7lOjVt1Dpx$Szsm1cV=E@%H%@g~ z{SPq6t+WHb^@%d!pNy`s0@wl5q5nZ3qd?)fl&U?{Qbm^T#@YW-1C{Wi0AmEYPy&w& zF}?&%9TM(;@M35&I1IU5`jIq^-@5PLSVS?NRvz-jMK#es;4GvagxFpErKHQXbnL$v zVaZUx66^(VX`(t)XgwxP0(Hs#B`&#`)5dR-wG!~x>=HU9A-kH`n?PWCQs>gi<%&qQ zsBg#`5$b0m@5Zo{E=Hk{sYzM+e0f=4t4__g#-*sKan52kXT8T&oV2EN34v%G36~!e zvIPj?cRAcRU~*}? z@N1Fa*I8T5HT8D|*b<925>Ma*{;Ym>SGX|j9Xk)r7&+jUs(*0aMx5ziHWUS0Qc-K*owU}E#r}%53m>lzehqGszsvmpi2COEx|;9nHfU_ywrx9U zY};m&#!i~XxUp^9w$nIiZ2P@!pYQMe_k8Bg*)wO)o>^zgPMx$N6epQ4**+O|i$UlS%&^;+VNGA!Z?n z@GD?&<+}gRN|6O%C|re{0RyIr2pXnlCT%+<<6kqS3dJ9JQvtf>sHDmSrGnO;wO$=Q z!G9OG6g6A#!H8_ol|}&d>Z+R%jK0BND~{1afO(6CsD`|>39yTeFaZny*0QT2?-5BP z1#+t(P=V}(`LK4T)Ax5+q>0!KyfPlfAV`D86}IlhL!^KEFY%s-&Y}=mFz@{ru`mlO zk(g{SrgriHw&vuL!188pZ}+n0QShQkSNr^`@hHq5s%HbMvv`}xk&m} zji;S?EZ+A=LV`Jn*@i0N_`m42zA{%0Sejzf60>o`INj4iSIoEVJ_*_U9n?qNuhI>{ zvBU)YsLm-OeQu$%!-NAUTv0>Fioh_`;JrMF?39EMcEdm*(4bLn#1o$Q=@r6SM-_K& zACl1g9+mfIkz+YYJ;I>QuK8XdHuvHf%i|=H!4S_0;he9IB11*5)BVp5(U%~?$8F3J zPnonI9&*;ZIZs{|hDZY+3R>{OZq!qrn!;rL*$wr{z`Y=QMXAVWIJIc|G1GRt7JU0g z^}$m!$f4G7y{KxZr)IY%dpWq=iyjOwS}DUwX!2U@(c+lZ=FEleoFACr{Le#1cBs%x z_knw6CA&tJjd1xM10cpxfj_dO%U-PScg0Mn7|Y{>O;1*xUyru(>UKcc`OsJsobWCx zP5S~?M9=xB>!s~A@Gc`eLqvc#AT;#dnXi>qUS#)oiN;JTi98-BqCdN)FbDXF8Vsre zf^QKD?*t*Tl`VU0sLh-X7SMTJ<#@$3G8K73&S=$$)Z7@l;~%1XUT9{gbF}?*xXn=M z59dJ7ZDp<&LJ15P8R{hitCkG#!`Ye0h|g9U3{g-+rVKAHi@fv1{UR8kX0Z zKApuxgMHk_@Pwdc=Q@?lTIftRllbZx===5~OAISFXz+nf$1P5Onz$aIi#{XcX+3I- zlB=M1K?sO%GQ#UeffQy1AE=pia{7LE^8Kwv_#44*(4Q(wKa;*euVwm z^{tXTW-Bc*#va1UW#6X}llUi<&wy*je?TqjiJ>#V(%RNyp#jFu(~!B6ms;C%L%;~J z6@Lxl4Bu_{W8(!Jr7JTMXOatQLrVwDS=tOF1Ij(%^lv0R04!Py% zV7z)ib&%L8j#UCt515q8Bf_@<7j!}bUs_U?nAu6K>nF&iL z7A)BAXb6q+U}D49huxqIq4v^QAU#F)qMfgYcNBa!VYB$|RRT|xWb43fL&;T*)8Z+1 z*&AHGSTWs=hPpGUBcaec(qd0wL*NJbddXL9)__>w799dekz7`!ihhs~T8S`V;QuO)x8cOxP^yIQB z0bE=y!Y*J^5k2wiXc*DB(Kzy)ApA2UE402guV==;o_?{qwCyox7;T1Mn7xJ2feV9Z z23(rE^d&Y$4R$>@1yqznmht<-8W&acZ`c-CtPY1WWO;w;YCX3R*Ar^po6T&k>wWU^b%7sCPwM^)-~;ae|N zifL+M{gFNauel=|Pk$2s*+@#rfikct+(Feb<)EqfLsskOYTapSu&qv9+M38(Gd7X7 zyqAD4`0Ht`dxV)@ds59W9hS}GzF^$!F5TqZv)^R8hMIZK2XE=@ICZHGmpka(%GbXK zB#vF)2Do1GJ)l&9SrkgXc(v-a1o0U_oX)F9p35fd~RyUF^sjJeAcJZssxX@)dh z-~Ezk`*+$u@qUIW7lv$ZvhAPc@-ujd(H*`*&o0l0rD?7wiY ztj%ZAd@9WIsHCV7<=1%Hog}f@jZ)NzxJrho%Gq*%`uV&ICq)23atiSv-tdQFM$vjn z7=hCJHIp(*Z^=V+_b$dK0fkV-5J5FiF6XbTtIIF5OtiV3+yqcA$fnPm#7a;2VzQRt zKb!AIQz|5ZtR}?@3iM$H9+q^f-%D0G%@U6C*5qU*VYVdh*{>E96bkwN>;18r8e@t1 zGIa!CE=$Sl`~8}|M3?WIX!`x%;{dsoXt^y$g8*|iqo&Ktd}+g(=nP*jK}oWe*xG3V zw0w(i7s7pr+|0{-59dSvd2up1_th)j<5>sB<;XM0vUXN@(FPzmHOt3epwk1U32zQ^ zvOTzY|1P%LLz{^Uw1=Aeeca2^A#$?mX5hhi@B@9fO3ZJo&U`T2y+eiYz}TnF4lg3m z^3Z$Ed-?GORmZa)Zq3bfd+}qkzm&{xYwYDYeAXj4#DSQE1auwX#x;H*NM+`CgUA2u z#AXARX1o%`J;2OhKX=11*NVqa!ILC!wVU zJ)a1J2VZph{UkAb@~NDM$@KS>2G7-pgDwV6Qr8tLKR|#WIF-+FVvb;EO5meEmq4Ju zTG=~CHmqGhFEtF#mk5L|gdCab7wv`Y=cLyHNQ%Bj3b3(GS%W>C?#(g(vbKoGpA41$ z9%!gZU8B;8552_ufKCRK7j`6K2C?}Xk@^w6H-9O6(50+gNE%05rR?9T`H&8&GY@v=&3lkUJ-=FwN~01$9_9kdYEe;}e^AMufw4;~Z8P6&Z@PAyh*a1z)T2 zn&3T!_fq`G<+*z7k{1qHfbe^3#B^MvEMIHJznEHyN>a5-wM#GoL{1T4g;!#sKL&kr zMlbKr-)uApC~4T!<}6G2)(4*V zIuudWQ`lo2B!etSS(~WxcepVFy70kk4{^ot97E9gL!FqOnwSrq089@~Oz=P!sLvHd zfJo){ePW`_&NoF!GDo_v;}s)+*NGA{)aluL7CWFh z9(SE#OmI1ZPG1Acv7ljKu6{XB<;Wz{H$NYjF&XxP{gIkpOB5(%@p~NRgzlZhN#F~q zsiC*Jonb%d{Q;k{5q}i91&Yv6&H^%Z-ptj3bUwy?sGQvZ3`f zD!U9KjPtc3IoMaNVFrn_fWR<*5rBANyY~a3`~DY?06NF95xyVv+;`u#PZg5MEAVMc zQjI>S;mN|U?QBz{pNR?lx}__Px+m{0G0p?VECZWd@Hd_>tV(zS-7w0U^`9MTvfyTFu*D3=Mx;U zP^6Ku*2MrK9tE+1i`4`lM|9s|#5tYqpA$CpdqTMT8^!2xxKVK6S9KNh-1Bl_UaB@N zM!$ET)0H|1F%SS7)%4cP)%mbJOnru$L|`+p1g}*9gf)M;?hIGKt~s&J zrcW2+D*5YKAE3UeL@{$B`ES^ZsK~~CVYWE_F15Shs6ER1uYi?dQ5G0#^djR}@DM!{ zpax6S;Mu%pAb+899&15YlG9VPvrN4zX^|9rmMqi}S@vwLY3Ps0?IalACL%J@@sIqu ztg2Jk(z2vof8`+@n1CZ69l@9h8}A!U&`45b=uI7f&I6y|S#9Rt*$kbEv@>$d!*2{&y&6Hf2A@mXImj2-9wZFZ=+T8>;gh_l|RUs!P7 zc31u`_P}ZBsj7*f&oW)0zoV}oC%2kN=kKEy1UNK%zk-9^62GGp6Pf2wlphE?r zHafyPXNbg>SL}{-A$^VBpqI%CcH&mXTa-lKSuC#vy<4V9V4^*;DORdO`Gv@CHmG@|Ry=J+QlkKRhLnQN;J{7goyZxS=0zOEUB|GYD8hf3Qo0sxGz%lf#XsJkt@W*0NXC>a zp%!k1Q)CJP<O6ZN&mVO_`P_RWRXZC)b&9dEi=kNUXbG=>O(lijnb&v;O# zP~A8U6DnzcgV&vbrN5oN$=0LxA#_tPhC-V5WciKPu~)#c)|dUv!se|qo} z7sQrBSgkhSGWQF_Ub!a4lo*>9;Q~|_jabOatM`=??xjZmgFG0*>EIHJZ||V3n8T`V zoErJJ54oKJONm>%jZgIUOi2O)p9SoOX_|r(_6(n1S5i8j1y6gLCv}4$Ny_3FDCF`fDNtkq>Cc(vnJfAxwVUUvP*o=vvY3G%qOcu!dP9z#-K3l36LA%D6?esWp2;@kj@Yet+LzutI0&tJ zuq*NzdE4m%;X0`otH@3`W!uuM_Hzc0Y)Z`1Kl&~ z2CE$m#*5YreP6#uMJk*H2kbjvzCXIxjBh}`iEo5`2BgoiwmDe~GPd3{74^CuQS0@D zj^ou9{rHu=uNr_w=eSb>I^#N+rNMF__v^`l$qsLp;Lhzmn2NwBtib!sBEb$pw~eqo znmbRS%Fg2Q%&FYRnKrZbbjT8KuIBy(fzkB$hvI58SuI=g1PZsBR(3lrx)wG0G$ctS zKSU(Pd$|3QPxyz9epJTVqy!7mPGjXO-c$!aUcpdo&*;>-XyKaf3`Pv`Jb@>wO^5eq zfpmqP-d3CJ^?{iah~R(p?x>2JJ~wc{QavGgvHaM;@^9V1o9N3m`>qj9 zcU)TFH4b_B4prqoNl2fEhHP+v$zq5i`0cH&jJ3bEo)@~@s(kl}zHfLeeis%&P|~_9 z8$Sqzt_P@H`!+W0@!S{mSR6ZTXs^s~DC9|CmsuEv+b7}1S1UB1sfg=%T>1vo1)@T$ zxIkCvyKp0}JlB^yI1qg8v25-o4~X0?neO+0=Za02DNPBXVqgGED1jI7S75ZDW&T7y zU1>rPb2^Iu!~JJd1CzbO=l#-rz`h9U2`r{EtN_L#hzLdzY#7d7oX(6XrO2*Av4$A`%5P*hqS^CeS zYGlX~?(43}uQYNk6q;zFeUbwEj+$Kup1T^YU%HawcIyP3Ei zMQ=#32KQO{FDQpam#G_^(bIU{odo{A({*AEeJ#3PtQsth*4{exVU6L>MJlgH2k?Q@ zr{kn!(00#GtSO2A%=6X8T_5mo%JfFQ@Owa*^m8R2%vx8nSp9@cMGppV)0YL4Ol3@cH3N16ziQ`HUQXDAG`szICkHI z`A^1EOl(&>yg3SGQv=Xw)P|>6d6R*6^~OAK1pK%Z84dv%fFTh#U>V3Yt7h&(t|ZrI4b=-W4Dm`4itmEtWAq+!zyHN%i@I9i#U$=bO`$bHthLAdon35e;5s@~GQ0z#~ z>mo1MZlc{=jV?f(?Q*yulFS07pJx??!N&n=96I07*-#1$x+)!>YJw9N-zqs zSrc`1#EyIWWs??a#6#xD*Sbxh9U~ft+;^+j3>z^0mDhDCLfW!sECWvi7@bSVUa-dj zY(wA1ZjTs^F#88QlX)Bgd!f>_$Vlf`o7bE^7d$(zkQ_QeWsonKVjW(W^K}{v-h19= z136CumWGh52UkEC-Z%&V2@qnyEZwdb(RC*s0C7SC>D zu53A)?m=yVG=s(nk+co*^d7gl*)dFbUgohT36QJ*_oaG#U-{N{PP5esg$g!z&C7Ktt~w9xgBD(2Yw4P&I#yU zZOUbf@j6aj(cJE6_uslUbj|HFl^aV&s>G6YoJZ2y;DXvkihK%Ft-+qT8*?*p)nEaH zVGi676%KLGk0vp@17es{RS#oveI8eE$uFqs+jS)T@+V*UBHzdz=l8nnU-UaY9O+4T zo3{D|%Ap&u*$OtlOITo@S6vV5jaRfz?HKcP7+V?%zAX3kIc~ZOockhpfR$*whpR){ z*{@vl65LfDjUDdk&O6aU1~@P|cYe!^0NA=yCpxs~tW~yiRMYdoi;$gr?WtNV*^gZ} zd)UBF%o>o?OB+|VzPY7|V%KO_G}FIwd;;|O%h#$Rf(cPlpyC*|v%;td`Mk>y4J1Go zeF!>wprba5F64V>g&z6}7cQHcB@zN(cZH2aA7t3Z0`-$Bspuy3x5oMHK-&Di7zARM zfe<3twh;%B4kXgxo#$M`ZGr3>{h?3kn9NBXZ=!crk(2&UQBJ4;LdfJV2W!_Vl1B+< z{>-Q-u|q6qMLhHVX&FG6Av5WI+A7@VRuo{rJZbKU{I3xz zC)tAX^udU3K*Q(}Dow(lrjQsd9@Hn*x>sHFm^}aBRQPMiYjTGD zSbPm_xDG`P`N*o<--kNV8E!#W0oH41^}Hk1fWKBGM|n^)DKRbP2joEaUz7eZEFi21 zq*=O#+)|C=e^&iOQYe61IvO*OLk@NP`$~FnUijTr+O~Ko%KsV(T{^AuD+~?%45I#H z6WDgwZa~D13pZ*5Z}?RFzag^>vhaO3^K?RgrbnmzXF8tNkMSGJZ`6B?dP$JG@}cU~ zExJaA|3*41qOZuBxW{_#jJZ@U`z4PV9#UYwg$Tq}zCK-Bkd zYBOZ$!wlO{*fafArT*)?IfctMtLSHdNyNU?UmGIzcoWSmshL!SgqZd(CpuBa{+kt` zBd{r9fyIISbE@sGt$-9;cXna4zh8q;2Rh^HkMW($iAI#qWPcv@l96PNN;@JAetQf# zqc;t8oIsG(-!kSDsmqnOSHa5fFOi2n_KWxDR_Fx0dY)7lgNd&PYrIDtuSO4lY6W^8 z&pvgphocblw?4Pz`o3oQ*pYHdrwnOPz+*R?p{b09$OjL;;c$q}xY=&7^P4zJN!`Y7 z55%TX)tHA+1P}1j)GaOXbtPuZEKyp3j6PhHJPKL{Py?$*xqHznBxihQPMKaoxVf8d zNa;kb3A7T>#1 zk7P)APOQl?b!cft83BYdoin`zI{`QH{Ucmu$^8WOlgjKq52dkqun|@R zsNuaAA%?5|c!jIOH-yXW&FMB(d+M^mr{zk#eDV6-JOV${)klSrFzIcC0R~z59Frj(gd}HOKm5S1=o!ebH_|=ucB;uR@yTYFFeTc#1DvrP2 zX;souV&=+=%*Jif+}H7pCZEt9?Hi61Sx7 z1}+5(ncQ!!N$GH=_UI&*>zt?q(Fe3@v&;pgfl(>4Ywiq1&h*Ol zS|jxuEMds-m$50ejc04ZOvu);duxDo`qH2YUqsglg0EgfhP5|aq5R2-CzAo0by1Ab z;kNGSn8Vi*BVlvMeEAm(e(h_Lst<-l{tkT8)rmOKC(AxZ@hEbbJ$zGTCv*Tob8zc6 z(eypJ?|T8grN64fvMsZ$GkFI_9{x*BAZ=!I;xWfBiQ-Ao2e&xcOvdPrH~Ks)BjOi1 z)t38^bCqyN@?3ZKTyZ)gma#9%x^jKT@hR<3gm~6=#xX6(Q+F27hAq60-L^KmPY5LF z_@+aPZ$WIQ0lFPj@&Nxa`>sIt#NHm2tlxKV&9`n&J56=k`v>V9x?+-_>Q7&|Z?w+( z1bW8>*4&Y8W48`+K$%R7ewIyb*U7ycVLOFonAx_}$nf>fpDefOvia&HlBH5Z_KGRA z5H#LsW|HFi)_H?Q$vTest$n_Dr&T4Bl zx9F!IZ+JevjCY`dS5(mi?Ylsan+);Bri=LSa#C4eVF%* z`CoQqo|oVa#~EN8f*}l7TL5y_Zr-%zSXbIosvpn0Xcc#MU2*UGANZb z_$uvz$>@g1t<)kQcBR`!n#ruq0SDGPV&xCl{^pLX)$#c~{EAl7v75!4#e0Ah-}_c1 z32#zJpC~G0nEbYVoBh1`0$q#MQs_l1r0X0tHH*m(EcWQvwIZVq^e`S9N_ z8E;GkhAiI9-?OfmO_~Ud$9@Dg&f>6>`5x?*>q6rwRnwJfnD>KlHIw*LWmWjZFPHr2 z6&oTa)=rM;fz_mEJ7p9ECQIvQ;UC@*}E&e4221W8osBS4U739Y_bAEk5i5fKvT+iCrxvP8F;rtf-)ei~ebjF2i>b zlID|hX{1D+c>F)T~0=GsKT7`w5$Gu6z_OF}GnH`5b=8>tdUhC6EY z9Hoy?%)95NTMs83grR)q<0{e({0}nq#YP-4uRpZCO0DK1LAx=XQSL#RZ#5VeLnlGN^LJB=_kcnc=4iKdZt(j6oR4l5a6yvq4tulnUG9gO% zbUC)3S(Qw9ue&(1`D+~ZTi0W#eJX?#;YN_)0iKlRTpZCgv;8>+f$|ibL zCAwG=5CWIbTw8pT4ppv`E-tO14v$czHx<(#(kdlehrE8ruIHtf7-&Euq?$IH=cxLs z9AAS?eB`69-?|t}#9RsuXk{HZl_wjkqJG$L<~dtyD70Ls)^Lafmqr9bQaCt%1s2SK zNPH~Wx$y>OKMPSfwXJ8`e{@2k=Bwb=)Hr1g!G%dM#n$1iPt6W{mxUxS z79PT@&@EH+8^x^WtL(BlU3Tu8NsrF7#(|ST1Mm{{Ga6f@njO8Jf}pr^kEm6foGr$- zi@3#;(R7Grg@!ITE&G=`jxJoQw#oEDhr!k1yGMZ*rpG7G5P91cY6e$3Iw#Yd?4l=u znpIwZ`G(hq72M`K^2EvAJyE;Lw|?fGdgD4CK41IJT8GzLwYyGBBqmx4N%K%@W%&_J zAw*F@KwDwnR`Q%9?o=1l#KqB1fNhcT2Um?2fXuww%BEqxb*J#BHCz*iVGcm8Y{C6g ztj4tuLyCCNTxwo8iGKD2@y?-#o7Wub1kaTF9gSQ$=QTq1jd-lO06l#b9{7z)Bz>m) zb%5z!Kd&1B15B_qfL-O34#g``FZUY|)Ak1diwj-0l_uv|=< zsR0JV_ux>$wr5LY8L|meivsN%0rrCc7L(E>#DNXo<4DywO36p@U+Iz)7Ma|A)p>nW^y9nHEWP zb9nv__&H`Bj3{rIwn{@ODW>h0WcM}y#jd>rvZ;%W&Rh!TqV({)e*vh@*PW7i zQ3@o{TE^%Mi%cp4gP|H_doXH1X2CY&Vcv^WGdKj@O9JU|+ZB<_{sTsyqu zMrJuv`DUHkm{!I6;{$twVa|l`reQH<;U<}iD1cDs*rPvw%CMalSZ_?spYP5R>IjI= zGhDCo`Pq}LdfW@cJ-GFEMy3d9K!G!gimJdrqq0btn)dxVh?;Us^4uAl0im~q6~c47 zSGWi)UTKV_POpCs2uR2~4mxJWZZN>Mg-g;Vjbkfi``ps{n&(tKER%ARC}QWD*FAf* zGwU5^fogdh_BBre7PQe2wG8CB(;r5v`lM5nksh^&TuBK*nwedMkYcOzvyEu4dW+vE zUM7xS&s{2W@|_PNi)MFt(d^+awK7Mwi$Tux8u069C?|*Z!6%lHu<{S0&9z7RIPF1z z*$6hGHjM#<31T`4KNW_ql+0Sm!Wf)Im#gE{SY?g;&keQh}s? zs})55<92F1s4*{&Tv~qM4YJL+a8F+SbppuX{<$N_h~LUs3j?!k@blSBb*_jhnW&WJdJRE6S?ajaYc`HmB8>*x?1o^!AS1&mXOzb-M4 zRjXuw2-Qj>PPTc>`ee%g=GAj()a|iN+n2%}92>4NuJ$66&HBQ#Lr4WKERyYJd+3Vh zatfA6J?6|kmS0fdIqDV+jrOh`o#B>(r@g!E$S8D2=aM3lSAwJ+xO#k}&+4;cU3B^A zy;&xhelOI1)pdYz(vr)zEk^qI&Z%{7>toyEP|A(2C|>2 zM5la$KoL(DTEyYu%q;L(76(q@@LS$6bb^Q7bd%=vk;F@~YdL@)H_t1dn?1 zMa?^~`Sf8B@b)X8W;++5RIX+_!mSBP*`Q6JJ9bs+PJombgdh5ls z0>^>vxarhl z#VK!!>_ie?F%FW4+ob83(c0|==4+qH;witl)dWdW=ONuaCFpPhR0|jDNq#1a=4;+bc4l|XE+t`$Kuzi z6zWOEzronUeVdSQ}gZ3rG7-hEM4HeA=jd;;fPMrgDRT+(C6(=;V`y`~8fEgQwW z<7+0?S%Pr42~!uH_`o~&mV;_MaD2ZZdX6v~iFKCSk?zEOh3`6dQZ8ALyKTE;&IMT~ zy;vC>C!nZ%49y`m<$Bs1NiT7!6t}j0S+Q22SUXji3Vn}FPS?10Q`x;*;@O^0g`~A7 z84w3e303r-WvltI;n-6?F-ocA?#;bKSFm-0& z(wZ)zD{=^E9MQlL1e~so?LS~P@V>D8ra?-i(0f~`DSQn+PgHPtC=HZnV=d04P7=H| zA8687U4!XTCsqX1S-Ss15cp27QrluM^|qPiYqo(;!KB)DdA!CjS7Vi#xO1&nA9H`O zg5XDJ_x2T?pfhHz6ho1bo(JkGw5CMD2C_nSpK%T<7C&8Xzd-#Hk8(Wc&$MusYLJDz z0GOf)$87bBLcOZK;wGzz?iX*s=dSO_li~^aIz= z2J;!p^;o-@lVw74wiMIUI6Ojbbx8^Mg)4iq4i7JP(PHV+jiE;Nnsvd%1`yBHz1|#a zi8CDcvXKkyUPb=)%)tN2j4T&bQQSVBJ~UU# z3YeO!WKSzW#gi*sP;=7%J&g3#=`5v}xER~bsVw?!S)M#9Z4};{&aUl2NYCe#y7wC2 z`T@S;a!GR}qa|uvrqQ_!S%EBxb0Kx%>g z_$1nVUn9NbXAIxJ+CuIt@FucJnn@uIvX#Zdt|fL?>Rxu`OqXM|zrmy0)U#5UouyjA z#WM^hl~#LflZ8bFGVq6@)3vLW?vO@(6KH+-fQbJHZmlrrHh|;@BPg>SGwxLlCrxL*G#5cNw}EYg%`w zweO*+)_;-H)7R3U98(a-{jd=bj-mQd_`v6&!?~~Y7IAX)HZTRtmm&6CX>1j!E=!=n z9ETu2K}xc8+(3;<_t?~$)RfsAZRI&A{L-rL;Wv-!rrwg4PQ05^Z@^(!fnv;@UFOFI zs^6aO^=`K~p4lD7K>l;j1<}H#?hzo7qF+MQ#QXN;;iQPsV)>%K&HAc$$bOEnicdzL zBbsPa&d6Pu{R;pY4=vq6j8EX}4ts6k{*A0}Hdzag8q0CtL(2YP@5=?h@b0mYXJVhy z`|~@DZ;EFag#gxgVnZ%xyC=In{AtTf&*P8=_(zSrr`KceA<`4nJCwp^u3T zm!eUpCNDH$OYgUDyj}|C@}8{p2ZZq<9aH(yN1PebxL9!gE}a3#*2%S%MtJAnwpCk8 z`3D-~vaatUCPZDj(y-SGnT%uC^Pu3&1dOUY!#JMZdPzWmQWUpk_3_P=;Rbs@arZUq z4NqrM3EY;T;4ifgZSKuCvC^zv7atxK#t9tu@U#c<3PaU*lGyC3Nq411$g{t_^gM6j zjUrtgoa1qu0#;s}plrR_)Hi)~I6WaNM%XIWt}ZfE%e9)abk6RmZab~?Yg6x@oqaku zf=*s;58l&{f5SYAL=IivPOs>AZuaqdIlJ4NX4OXOcj8XYS4bV-Iq!TGUPZ=~ZR4N1 z7L9$*XGj^rAn7tPY(!2d)iHU0rvFwrzmwGV;w*8YF%^fh_%<*^Lhi^pbnLk(PPSL( z)|LLzR#o)yZ2R)`G{%rx%N6?9*Q~b|AAW`@O@B!Jn&WW82udQIoa=LR5_9dJ2%F({A%g+$4 z-Y+|=aff`9{tcID(rft<$i9H>x0+JjiIV~bC8IJ0d(6^Ffjy|hh4^2n5dBpF>gOR} zqzx8T54^8LTz@?}bJ99i))=vIGid}IEwbPDuNPX_J?}Y-`@M2@q-j?`QVUw~t`$Y?Oki|v{~)Qd;4H`Gcln{~Ja|1ZAB}$^DHtZ8DG`Gxa6zH> z=+rqX0I=%EeQdE=Ugys_BkVpSl%@&LOvPPoN?w0anG&$?VzDC^34t?ze+HOPa~VcK zl3Y$}6)L=?2dCf1=lSmP$&-h_G}ir=B-T6!0nn47e*FqhGvW6`#X3`LHA#Ne{~K)< zcjCZFEqPMe_Bew*>Nr*W#ap$)dsFxe>~mNu zP2-RA`4VmB-C=3t3;j*UkD;y-^Bwvg@l=rzZoN9Mg%TXko%dpTj?3#N{hM{?WRz^0 zN@|BVM-vWh1l+B#b**MpE#>8!Zv+xQojWdm^2DcJz#GiJPT-CyE{F$H_P!+yN38Ys zBN1HAT$6&3z-rLx-DEBfF}0kTXl5*nbdWdSs-k%KPA8YIe-lh}%4=0XSqX>GWb(y) z>(csOlyVHfFc~;)C=$MtNBo`&JXaU)3!yl#D)S1*O7&DrBUXv_ zPq+8?`v70i@an`MdrvvWrZz;L4=A5x8)>p<{hGUS|rg~O- zPll@`tgtXx0nP^=)s-?RbcEB{gDtA84amC|>SvVeAK%;`~idVs7xKSy( zEO_UmR{COHlo_Xu_Y%2exJv7i_Ynjyw9l}qT9)TZ1luYt8r;m)(_X}lsdSf)xc`#b z5pO>K4Q_krygDz+co-e zXBWjDRC^uy@U->ddAK-k&~^izrsK(=?`AoX&d@%V+bnDM>H`wHa1US6tiq1T*0aOz z-KJ1!V>YEb8c~4-wrJl66Ud*f9WAI6%FgI?87jNsfY{Y8A180Fm^&@|eJ-f;2bL&K zC5s`G_Vj@!dINWF`7x}!pqQ7yJ?n$JeW0fOSIzeDE|nq!TQ%@BzDE!e{PUPaR{J#G z456$Tv%NjGUkfE{HKrNT2OG?(1RS_Mg$TK0pr9VRllP@Ybv4Ij?w$`z!Od{uJ|t1x zcc-_<*e*;JhXPw{UPB?&Of9j7pqqQK3lb|l&Uh}BJ1qoL&T3O51=zRKgX>R+4}(4e zd$yfv^>G`ZUHfP9svQunCf5heR5r^gf{F6(%ns8itSSTS#=9zC;PFRR2l7%KAHSTJ zYmQGRkca6A1$_xp{x#fGBMf%8Kq27CwAyQ+Hnnb`CR{Z9Qxo`i%XCS*V^(DLdj|k2 zlaN8Dscr?&_SCPlF<^ro*T{Y9qz(>?TCn!FduL2%Pxt%DsnzV6UT*@%%W!P%T~pCO zA#9-&>xxw1h63|tVtUx(vy7kL$XM2g{QVAr=4=&>uBsV!l(a?(yla74({-;*(~KgN z&&1CLlVP)ayiu~^H{&QQugIa}UB0=bWA{>tBP0bf)>(cbm156EmB68_paZ>NEyL1)+f;&U zE9<&dGg{{SLb-LO66vwMP=E>9_|Af_YKxm^aJj&{f9!mX-BCq<8qyL?V2=H_J9q4U zmw*eup%s-o>Y+|h36+Z`B_16Cg%BnC(zjA6&P$Y}395}cDpd>aZtmzLhcEAV*`sW?0>D5|whn~J$LeiU2U8yxEBl_F^s(Z{z? z2koji&ZpB2V^av>TBm<4)f*YZ-`+a|ZRn_gNu-7-`v$IOl}@txb-NbhBF}hwwPe2I zG<;3AR37i3=YX$wL>a7%bvKD#x45uh_1w}dh>s?E3%_Zm$z4MDP1YhL?GbNHqxF^X zKCJMUyCz<34B3w#39>E=U+TkH5gWW=AXu13ePV2HlH$_s3ckkE-#$Q6z_HlkcVw;l zzdtc``doaMu_wkBINDj;HP#N$*Hcl}fmgb{{7km+|G;v{X5FGEqEm1H(pSUI-LU`Fc{z(Z-X# zDLJ)OM3K~t(C(Nmh|bfxjz!ybHjWlX86ygZOk`Lt^Z;Smk2j+=$Q&Z3)C*D0MinX9 zR}xAaobaXw3C;AG`1o`6(WlW~YaST6VMV<5-^L%f_5;(Q77!nyV3qQD`E_&DjpFWk zt(#2PF}4Ihz_mZtPfs$_)T1v1+*eL(2E8&kD>X`=HdpNBE}2}ZM!mcqXzO7gwf56E z0<&f(r4Ee&P7>H^v7}Y@GWH$&C(5}kG@D^UXqdrl!NUeUBi3#P_HMc28W7>JB?;xC zDqH$$r)=!rbXyhpfQzN--SiFH{PDZ~lzu~s7p%W&Um%^U6_SjkqS(gs)wEtVaq%?b zL@J<~7VcC!sB_*X^Dl-o8bc5T3tgZ#f*Y@Cr_~&rF%?mpa5a8avjc`FLHp3?$zUhH z>5|}CJ6*m^ZhPmFrbFiWm)nNpR;Nh*i{w9VT-xjc>K)9f#L$$)rVp& zR%bqGq=cpqwchP|=x+z2oO7-G3MGVC`hq&QG?nFB8UP)-Hc(*H)8bd`2Xfqo7Z?^rax-u1Gf!r778Ld_b?*)^u>l&d~c{nP1!5()Sv(5*g8t(%&>Xx1?EtYT767O0Kv-5j|%>F8`>-LxU z)3!=~$MIp&?o4g2<)ChSCFM%HzwC)hJYI~|+9PHD2?BC9He~-Kd%!34M5i({o(_hM z)Xq5r$W_dzOg0>7oG!o8dmY((b)5tV`2EQOkzZ9P84k)>b4!DSnEBj#krl=Ka(o-F zMLE#~Gr2UxoByv;&eybXptG7` zU*Vd&lh0bT?yf)nB1d#S22rYbjKL+n-)-HcF4}1@}C3T9~xROU#5VdG8rguAYaOi2Y zMidigdl3W-m9~U+J&C&*!`aVHQJmJB*}88XT(_M%SzdeJ`nbzW3}$D1jwCdO7iwh` z6V?eT&7(1^zQ1KcL9_hLJfZHwY@3NtN@mf(g2-)xSS|_i)82&iH0~Q*f?oL(!j4<= zy2PZdX3CtpePX#Rnm}zaqSv|H>J6^#6J^1^y578T@};|N^G%d==fZtekrRDT&au1K zG6B5yxKtqI=HJ9PKwsFt7R7k}dB@SF+3P~EiISxnNv|tWMEjFkpBrPUl~}5oH?$b} zJ|pRwgrt$>T~i|buyN?1Ubbgi>@sDY=igarweSstKQ%?hWY4tg|IIg$&>@yH3|W)t_lknscjkOkx|y zZP3|;>lH}VpBqe>0D|eYUi5nJ2}%E+20nyUgoG9mO33ZQ22yHqL^K?L48n_A)NDD< zb_+FyE;PCd<<(&uMyR5+#ceWj(UR4u*))E&yi0ug;Ie?`=J$%-UYa%QP8-U$D49LF zLKv_?zO`{gg^+z=!~n5Iw&5yz~p4nmL+fsUk(gB`49Z~`DiN!C^J-Wq8&?$C;Ki9hz#{xhuI(&y_ZuRV}MFzsyziIYIf38&I&<{$2tCi)<$k_G$&{81wUfVw3eI zOBwGaxdYKYBO{=Hr*VUoi&OAfQS?s7Ova(!PDC88rf-f72QgLTuTs0U;F&ZfzJ(h@ zM-CmVjS@U+Bp%SvzsR=S@E2k0lJrpbVPZ>rCQ}L{yVu`HF5Dk z#Jattu(+OT9W?EafnXu&#lWCZ=eOb=%)a5&e|X(e5cUy7(Pamp)60^%Qj1)=?}bhD zo7QSiWmz>!(eGuLE9{OZxyfjy1DL(bcbv-@`25Q~nq{gi5GFHB&mGsq0@Nz@#5n4; z;ts6GuL+A5ep6q54#)c%0*0ylX%Se~A2i{;+Br*Ax^zdak4 zDiq3wbDy64rgXbt{rr2%@6`a1$pd@qP!Y1s(2Mr7 zx&A}BpFcF0_exasuZ)A+#Le@|i?<~A{Bb|0vWu-#zu2J`Y$!=x zRI*-FPhFdH)-7=Gp$y^F* zd4;Ha45MbH=)yzqeQi0I#xBQRZg-s_@GpO;9S@&?)pP1!5=(5H4>cq_9JLZvFU_5B z_8Emko0ez)s#l2-%e>E9%O4FktFzvD&ngF0{G-~cR&t3BrGD~d#`Qgj-dZjwxD$J2fTU|&QFq+)@SFv@tELY=o2UQq$ zdQngN3nc>3nzsYRaFt-IC4L|DpY4vcJR^Psh<@m;?BzrAWQlK9?;s07L3|@padc#a zL3d>tVB8uk@0;}@7Oo^CZXxN3Blo8rv;$uKeTKM%{0gG{_$G%*Dr zbymP6y?EmlEeb4e#hZME9cKA}WQ|436X}ytyTkBS{=xUXc=j@u>zBMM1vf zsJMiIfHno+zPDLjXn5G~v}Fg5AHJAgx9SM1t(1{{x3-gqnJF!%uwVhdKA##{`;~&g z(~^4k1rFa%mayAVKhea$C^C&)%qcqfy`gU+Bg|iuKSq<1CoQpVjGPOatj`$!IrUk2 z8F-%G7vlK2l+#9flg!zdo?5Ww+YwNw)Xy7o`!3IOHC5-MH)6dH=SSO`wBK*oTt#La z)|e-azK_;nzU(QZ9c)8Me$j6>%JaUX)$wc(wy}Y#Y*hWHV?;*%w^I>kQo$R~WopGF zew!Q?&qG{Q-|mQmRdFs}(E_naB2!pA30y+m>7UCiNtz&^74vn@X67eD^`8HJFfF>X zz2nK1S7?n+zgeqpDp-G|rn?JsAuE{Swa~;C*la?Joq2_3HleD0` z;sG=N<{^TZr&*TYtr)?%M7vu+g#BO?1Fy{7nL!%23wf;oJ6SC-JF-?M8pmz`te-}4j-?u~a*G)oG;H%uuk zEKc9hLSdYofG7T}LVxp_3TQxo2D@@F*F0N$&mxY&_Oq@uF)JaYCW0Y$FoNhi-q=@|R2 zHGx>8UOAKgHKMJb;1iniL1aWR7BWFVEK3G2qz|lKO=q+eO>3Rm496C;7kMolR=-Z` zO6gZ(+n`q7;Gf4?R;I zyXbEdB7!c?#l&0t?3{_14W~yw1t&E&EeQom3caJNtE*u7(ne_|*rb=%+e0Rz&FeR& z9M)Wl@WC6bg2k1tN4wT?R`zFOK)Ph`{k`sD=PS5N;RY>s_@I!vIr;wneuaej+pWC+ z9pTpUD9+P^n%%_DHCbtPaVR(@hy=V3xY!}N8NXa4ENJQjjtJI5lB;ZMtrDF%PgaJF zl+-9Uj^%(b%_JJN>OTiq{ShMdnzga{D%-gEX=9jt;b!h8JMF`v57**)t8ti@64L|6 zb4D$YNUN2c6SX)U{Oxidt!6KTbpPh|F1P2y6fPTNnpIn}Q9S9#SUKqDDO*{SuMRqC z%eAuY6K3mwDFjkc8l!7cEi`{lQrTWgp0flJVXo&P8sbQ6xt@{=N{+*QON_89t z$ofAteA>+L*M%PAv?P|;OP^v0Wsk^}AMsC=&`B*_EQMVV`5%?w8$K_{rCqS+Yh_#i zL+dv1p}y3pF|I}aLN8Q;=EcibvtDQry^ zQFA++Fuoj_#^p_svdx?s3VZiWX;|y>X5)FG$VaFWyhg{f#I+k|=)b>AZD9INo3LT= zzwmd>pJ%n652_FX?=) zj?px9*ar*V7_AyEq}4+e1VYvAj_7D_56W@l)3byWLqQs=v%;}>&fvsnH)0KX^JV>- zzPPgG1IRh?)Eijbz{2q`upvv~PR12V4=Wa9E%>pe;}FM9gb?pt-D_&+>auTl3q(8t z;wlARv|$e0RD(#VvY>-<6%2n|NE83x2L9^8R{0#NzfX8d^YV3`a*ot2Q%(>D#EkYRdfjVJPCrY^US4-gXSYG|ELL z#h0Ci?Tn6Bs!2J8&lRg7^@7Qwz=b${P#CJOziR}!I!C?Bm=#-)RSo0!7K*i5eNA5``=9!61b zRY|TD5}^sswg@s7cCTO#O_W5-<=<$$_Xlhgx!~jgk=~!IgxSO}hc$WMoREhQK$G~K zj_6qsm*)kSh>ij$H2@8}*+E{G%S&zRJbb;$Pk>5_iqj39_>!a4N^aaih?C&@dR^W} z2!1m;3?TngAVu>50XX=ej=LGeG@IX>s?~da_5CExGFqx>*D0X}xJC$kz3*i2zlOoieIyiUz%4GHZ|KwFz_o^Ak=cbk`l=a;I6LvXDMV2-w|DQWg zdgcElNB`OI0LKKL^x4IGR^!nBbE6#;%L*}11_AU(cyy)%C32aPl9GgceA#(kyD-7c1hh>3#$ z(UR`h8^Z&ku<4vm#55W;N1+({c_}S7dR9SG*26c>g&ZG7RQ6xLT5(d@!gcNPl^U^v zzgb1-N{5)-&pKrzIuhsOUdcCfrQZi_Fxj6}*x7xPkiO_gX9>;zbIi4Ia$YNf^M^a_ z+Vbp1^EhD&AklR^XFBwZ1E3z~yS)Orz7NpdKCdPE-QM{3s5m$>vpIZTfb2OhKx81V zu{9ti7njp9FNvg-<)s>Z5ZjE524hQbf!Z6>+XQ* zIf0;%&WAA^1&hhHU*`Y}_-*e`&uKe60sj_DlScw>e+}K5?KJW6uD}~dYuSKXL%;Bf?Q#6&FZ6#fccZrThD*t80dDw4b}N$w#(-xI42#wC12jQwVf?^qa}P-ja%S0Q84lJp^0~~hUmlH0E zDViIPdy$yDXs&+owaU1^LhlGnop41Uafue^dp~qoapk6Y%AhltHazE!yZU^+BgGWp z=nHZ@2S{rBExZy0(^K1N zki}%=7dnIfRI-M?Rsm?cjLscr1-qNeYrVxeD!l#=s@h`%vDqZ6IsQEfGc&4Q1#Idf zjjt{iWE$&{$q22UlyM4P=3-pMG|$3U?5I~y_$ziSecDiJVUNlC$d_qYI1&n`m||>T z--h>eS0bjaSUBfA)|8 zi2?b=95(qEXOLDowa|^c|BnrEz-WIr!S_%T!N}eFHQrt8v7rmiu$|(<`IvVY{QDoa z{uoR~du$eS`J?=a(KKc=+``v#QFNsLJPiZGk)sc;4z{<1vu=kuoez((ug%%}sYIE$rP&x#e_k7x1&cHE#Baw;Ie zyv+nJc`+=q5akQFTsxFsQ&Uc&hyRmnQmnCOfq!N`Fs~gl*0f9x^pTmgZu0fW0VKG? zH>)|>=(LZqJU5gpS^}h`Lx}|K?d=qNaWpPCEq^@bPDfyP>kau{pRYrAcYpRAKNjy- zkfI(69LzH$k^RYi-7Gab zF9`5$Ngs~H@cAojL>-=VS( zW3;NVHTZPvnKj|3YCIb&8n24OgOk9YWv_EKcd4%0x^uFnl9RQ$j)MPHiW@&#mfR38 z1YdwEwX5xUvbG&Df428155iWXKyT0>M1leA{?g)^a@EsAE4m!kOJz>-bY#cNL9Qhu zu+xnB(B&@FIr5P8yvfnze^@P?rf;QKCavd({AhJrq7D4th93tkKbrc|S9M+F@^?pv zhmp|Gj=k#TW21D#bEsoB7fL@wd*1hP29hJJes-QMEv6n(Bu>@eifUy6Wo_R^6CT(W z^M%sqpl2iK$hyU&gZ3 zhXyQ$tI!{c|6azN$$Gg04TW6xE`*fwY*n@0+04GbmsSi=le4uI^xTT`N2jv#@iHW z)##y|e>1(cJ*H4z1%P=hMF*UI_2qfIEEcFk`J~VrE-Qfw$+=qY7Yt+?Nc@5 z=+Ab9p(l%Q)!D;k#3u(7@6v0oqQvL>iOVE^PL zewCDIQqxy6#5m-dyHRsvU1mj;F03>yR>{hTKZe*LSLB%jxP=n}Kc9nR@g!J)SjO*v zmF7+h{ABk41dG>$FszoM^PMt@c`#NK6RE=+3W*(eQ{ixQx@y-xxADUY=E~=yEIA

7`We%x)c%zBgECc1uET_E_yC$0CP}&5M!cRKe)+fw7{}rh2;x z0w+oHC24jKoA-XEl=J%)3nd`4uVfFXq0za_?-|r}p-I01tIz8V*y$VZ8(7lIwvmVD_cJtQr_Sc7niu5ZVS9%747nS}U=M#|Bij%7;OZ^HP7f{9!us<|(G;<`2_ zEjL=jf`W|wd#CzAQ-?$W#kj}$V2X%>1fw$L8u&ZpU90Qzw7dzv6R&iqEY{~6qh-;Y z&e}{XsubX%cOO3CBLoED^Ofn{Dp{e6e362spnC(JkMqwh1V4aXk`85IlJ^D*KV`#w=QCQ&*l-%3RBhIxv4zCFgaBK^1Xih@%ZD(UXGSmx{2_7H9|;{m zP_wMS*Aogu*BuCe;&eCUw17K-f`Y=w^1y0TtTmkD|86yl?J3MfZiSLaK5KCtc5RyR z6&ZQn2iK}6GgyJH-W343YF{=GwX0y_xVfjo-$X)}UWmn|IZzP!E5TDc$gzH}VZBb< zE=Q4}X9nFK^jpXq{$^(;wn~xSMltz`?=sROT&2{X4rKNm%?!EKi6Xs!*;0)~nE6a~ zII%LRY+nrExcNOFIYTN%U!wdgr)BR0Vrwcs#;kJke=tw6?UBNCuS~7Ct3P)C?MyV)C6 zD*Wg3i1dl$zTf_hpnSWh%8&WO^K&y{dL8~FQK$hi*J>xG%sy*&lzO8M&BYquzGE%1ZR(HIy%DA9Oj$i2*CpwRpAp~yoRN@Wl*Md3noe|o9W_}JhsKb_qpVWOD za96|r;BB(wl97!M!)fTrlLb8gd0Wl9T_jNIebhI$5v?0F*YK0Y`<5=m-RcTrGMLU6 z|M9WHGPT%d$h|jG45+zXPUs(mV03rKeb#0-TukeKl5Kj>>K09KalS*iqcC&8bwSC$ zO(F358gA9{ECUHd=b^#4Cb~Q;<<&KVZS?=na8oVt-VWn!w80yM+#T=4iaAr~3&t|= z8*6C}$NG65EW%%(F8G2_Q<6EAK%7C~Zd#qi#S*y|ccZjDt#9zV2cSUIk6-ZR7%h>S zrp4i})`s5QGRK_L(*7-iW$10D#wu|QO@D+BJB8O@ewHnF@bGOck-D5}pVX{}3OSH* zEedw__*ot03Ot7|*;S_XnDV8&Q~$(j}kIRX~Kd+M62uK|foNwUnieu@N<3T&ZMU zPI^z&O!S2X{rDRi7%G!JSn)9Kip8`M%TS4;U(LU5-}VQ4gA};bL%ibU$KM;&AA+pt zVIjRqV_}xyG%)6By(N}pfA#)bC-}{MY;p?~N%s2E;JDg`2hs5Uw83NaPVLqZ@VQ4v z(kz^R^qpYDv zvs>*hwRjuh6ozjJZ)FZq!w3O-g30cDO!h0zvN&w0*Vx!j` zZSz4P zVoQ}Opc5h8njKC&3sWGh(!kE|P_R?uQguh;?Nneg^z^#pq8*_x#knK9JRZc8t<2af zH6X{Kg=-xjh)ZZe`RiN2DMz9adc&~3x;{`g8)Z=scAcak*t=qdiqJR;+brP@PO`$yXX7-eiQPUtSMn( zXZ9d)o;@AtX`Q7&B`V_6C9Jh|X)_K6FmF#mlczv!m)E-}oChyzSYxhJ$8e|E4?H#i z+vkCKztrU3vrr1$YO@Bd@qSCkhsp0z8O-LG3HF{QP1HPaB5-|A-gXS8ST6}vO}qrn zEu*P_?5@P~Xn@LAhY8Y;9e4PT0eS(EPM#2?RS_eK`OzQuql_ubrB^N23S^-9lHWR( zW?)&`&@Ay38pc+MBN2JABuv##21P|#LmIcTqmMQodzH;m^K)aDF~NtsVT0CTn5rn( zI;+75lVs9gfg5|!bhLG;NcIs{Mp40%@Xy5LnM1)5zt4?g;zkTo<&o?2iBOTS7?MHJ z7pCD0DOiQAe#l^D)lnX{gAx1C{gH(3<1EAWk!*L#(qzC#=xesM%J9%<E!d>2gG!9jAz9&Xn{bJWlqIk2idteOwPyVpL$!7=P=!fQmI z>F4?xWjt>em)yPpC*}T|SsmhF3Y2s}g!_5HNBfEu!{0o3mqWL{*Hx6Vw{+AMvMFekkH32Jz+tZ^0Ey6_qBwGmgn4Q(z~Z88g1e7n}fc zv{yI>*PSxv8>$)`_y^g?d9p=Sh+LBx)*>&V4W{bt?XGMa+7Ej*B(PYDD7$JIThY;+ zAj(&Elv`Sf5bcRpMa&;U)|J+UXiO@m|J#xUoBbuMLa}r)K!SfE6eJzAObuDfr01w) z>Q_-l_gwky;MnBsNcuHSJ^OC7gb#a?WSB^KLiLraL!wmZ1RORI^^`$FY~!U(dF`Jl zBr^Gah~y60PrGa^5G|;085+s8)5{ZtrorBVzB9O^K5fi@>pl=?B`tzYI z`*qZi>wNEZeLwFxBoQn_Z%-8jA7LOUm(3~BXsvr8lJW6=zg!nOx-P@`Hoh`< z*{Fr2QXyPu0nW6PLCdwNYd@~WzV*RsduMe2vcX8MnITx%#+-&X#! zyizRsd;=wq$^D%!v$~h2M)Rghr5PkUCbH5eu6Mui7lM=90XGNlnLQQ?faR-XQJkei zka%(EN4P{F+#J|7fR7D8VUGe=e_Y-aqtYll`@H$fe}(tzwkkwJYP=e#GNycvs7Z&5 zMIlejeh(wdMYZ;^>Yg=*mPf39K*_RRhKwX?+J)+*`E{zFCE$muqh8xz)AsAY7@Mjw z1l3HZI4m<9#3D}oaP3@W?QwLMQHn)Qdaai9urK9#&5a38n=iYtPDD}Du;AfencJKm zSKL`tY9wX{39DpiBL;GOmP63n1UN3IKk~XOoVfx6%*nOjcX1-8?*p$gp7aHanqpa! zXpba6S1O{*!LJ3_2yZQB$NeWO(o^K~26Vz>W7>oU*gJ(H)$U^}p48)VS9>0NuMQ^7 zYW4HMqfu-EnP2AE2+4;TXJ6&R^^S7A&!AXme+1tc;NZ zMPxMh%CTtP9H33oJX7pK?={|zxGgONVYJx&Rx$c_=2XsWcXV}R<~6&~9I&X4E*_wk zODoPxkZVtCIt`J!wc(waNY*}>J$4_cNf2C!r6bz}+vE(Qlc@{wCLqx__;+`6jcoj$ z<`;3EPj=wE(Z1l6fOP>LBaXnpVTVOCku-O?wSOt(2X(EA@pc&wPNzk9Cr5;{K6OIo6lMoWbpB`x6QWcG6pRhT zdRgRb9#fao2{lq0So&6R3Fb9~jRghE%lscA5~?W8!!)@SzoP4!5#3nm^!7{K656cY zD*_1}9h^iQ{%|ZF4e>{E!IqH3beoi86V5Z3D$)*}lnBr~m*;*lSKV0V`cy%C@5 z(;!jBOtuNXYXT@_QiQNqZT;^!XCv~U5# z0|C{VSiU%kXVbCqd)YPi+xwJ~jgrH>L>z9UTp9&FFun*G2MkIZu4$Ew8H6uGn_C0* zmOg%0)jOpj$n?Rg6K60^;5z~w7Cle9+1m3o+k^CBC?WxNirD>R3ad)99)!hw>aX*5 zy(?XiBd9Ey0CeU`m6)qDeU)u}2oWM&$Fp!HR2+}WzaUjxJx26gNcho*jd-ZjqsU;u zIOG?PTz0SdDW+4RpHxXFO+uokAGI+ClmQclBgsDS0A-qVa&2>U+qpd83f1x;Hk*~b zNrv8}5)1sUR*Q5ivei#A)ECh|G{=vlr-B7nrhnid- z`~^|98;E#yGO-se1C-6h`;OcVo3^o)P^gVT^=L@!nZ7X%3`E?wMsz$r2rd^cHy#mP zet>yx^%{Qp$;;2s(QO>G@jDrFr(k)06YL?4{LC80?vIa;dji@l(wyZi3$AB#L|Hr@ z^3K=+0E{q9>Dk)PX^sy(iFkD1$#kYtr~7pS^mS-&-MuO+D^lEEw*`0qtQGo$Ef414 z3f{oyTGU#qtw^;anQW?p(e(hs{dP;?7&;r2)dChtID&Ygqd+^7W`omSYq4gXDcg*B z)E!iLutY9in7>`6$mEqAExe>Wu{%RXu)*q2TdHQ0 zFD!DkpyP+X=lOc+9C3)4Vbh@RIcv{{FTVclhMAW6jH3RzoI zgMfn8cJ$D(z^W%=b>PFnMkc~O8(_w8G(PO3R0Z*!GxFmF;+62z-8 z8~&|Qm`r70s5(0^bQ{pGfWu)=lF3P;rNBll(|)ripnlzh5(S^!m9uge6>sUktc~tF z8+jN>oYWUT<#Lkb%mqJFFxr1lbgW##OQeVsDGG#_+=^BjBsUrN*o@$94xY|}G@Dk( zBf1P{PbY_IKzn0i*j6{%rE=wKIM3q@F2nLsmFd$j9WlY zs(67-95d?sHfO_rnPBeW562wYH5a{LV{-V~sGJo^d(XW3OO><>xb=iCanhaeV{G=!p zVSm{@&#yh01d)i}q}Qj+Rm(Im*g&#K=P)Bql*zG=wTlNNdxS6FVCVV;0_O}=e${%?Gf?75KroFawF zmnPzZOa_IALw2~H6HiY!7+MvNqmURx*QK;bqaka=lZCh%j=s6(sf_zkmBJsYS_2bj zwEq5i>;89+2h609q`r5o4Ic3{Y^WgksYRGcR<`t>r>#fesgd{K6S`mjir)7#nylG# zL2*@gR(H{AFyCbc$;;%7806_f1xb2$#PY=O3u@)ZrnoMXP8GSruo~Ng`7ztFxh~`i zkrlfvsjG#Hlk;2T=v=(?dIE4I!>joH!JAAf;qULS{7J5XopLY~F`gBV+>$-)M-6u( zJ7X+V{}|=MCmjq1a@s^jV3^oCdu*Zj^*zX5>sC z&IlF8(e?S)=bKPN(wfro4RDm+(D9llH^-D$jBeJNA5bddA9}Zmh>;oMA!Y#%p`?pl z22Hqdak$+a6Q)8Rv2ouD#I(g6JnRj7dH$%uQ#nhfGjP`cXI2}|6CVRn!i${o2o!XW&`_ZVD$wEMXi?~8iBihfuT{qrOia2r{#g=^?ql?ZqfgE zJcePS!+T!bNP0pa!+zy$IY5L3z&LqM>-3aNR_UK*)|p{19jb%knfG&7vV6xU4UHnG zJht>+y#-4q6XzkNO75x z0(BxR{P+?Qf*2RR|Bj0df8E<_gCwfY!g=o_ zJ1P!es7Aq$6%P4tY9@H)0**d4(t0^no2Mmg3d1f$%skM%J+O6?Xj>*6xK7Gm>zQcL z9MgSf&!EpDh4G9-eYBpzBbOHadR|F2qG2oXc`1P-BUyG12g3RN=-tO{=3o03625&x z{eTnbu0~Td`t(5sRykZSLRTv=`ba}pB{8g-d z@pq$}eSQ9YBym84IUpTPfdx)@P%<1Cp_`nd&#Q%ecW0D_E3Sr;K2UlPB5SEY@w3t{ zj$4~4r4K=-J97~WBZsO7#;We2K@9o-IMu`14B920Q?;BIDELm zdCeImh$lQQ)JSz6b%yi^V4RA=Kq4FoV`ap(B2fH*Yb`JlNV#nHGwdETmr;blRebNY zCN#xfWyU25h6MVlq9FA_9EInhB=Abc<8&dYazjyLVRCAA%h37RR`9FwvDuy=R}PhU zV)7WnnQ6fcE(7L_JZFWjH4bbRjh{dB!R-0kriAn(qJv8L?5fq=hk;`)o`?Ob`f(rcT6N6-GcGHE#=@Sii{kqrzCF0ku zEn*WnrZ2V-6ofUrIDF#6S$iR=yU^HcHeAz~`7ND{pfop-Uw9F#01P5>o}(Rc;tF-2 zYE_7|B8?`a=`SjQAJSO#`2|V@Ma_;NH_8-6690$BmCu6#N}SE-?c4)Gv>9iwJQX?g zPl;{E4H>mg{iXTbEyE5N%!ZIsBb_bXXMWWxL9E03{2_W~Ba2F=Y#y_cS2fbUqD(`c zkLj_Zj_Sb0A+Xy7IAM5?M!DzX3U$C3CMp*e8mDRu_`Gkgth@6}^Tn|SB&%)@su)M8$>y}GpNPxtUxZx-anfnYtf7Wf=g*l6TVIbY-hgzb7B1|ifC7j3ZZj>%?+d_lksnJ5gp zI|gBiTt2Yl7LJ5VrH~5l>IIzmRap1Zr-fQq&MrcT?S3x3I`^0PQ8+Q&AiT34-&a># zr&?l~zmRDKKM=7TjS7j)7nfW@%t%KTd5Rq=!+je`mbAtMoZSs6n%`mex5XnJrxz8D zx=_rHf4=2*ISR378in#%j1<=5=|Ang4>q!0I&s{Nk!lT4v{&0o( z4;tceAzq{{Bc9ZPx#JZb`sLey5)Lta5qABcJEliO9MIlLJCvEz5rSaA9n*U6bTIV2 zc-xe{*nyz?Lf{sffGL#mAm@9%SP`F!#`Aw6a{2;aOrV|p;1hFmmu<#|naCPYcG(XY zNz*-&KmzkgfapoG>v4kyTmvoGAxd6x zHo|fu!?}I}l_=zdsj@Bm1)}(rXPp{NPK8cP2DduF$ND7mZo}wSUjM10%i=_Xgq<4> z(i<5Wk>5ZLCtj8*6&Cebt%j};RDZyWUpgbq5jI#^oU7m(krYT@ z+`i-8seUn!WvqZUcfyNbBb&3~QMdYFCd=Sq&sSjhO;Cl=L)j5{{<8CGq$F5xH*=`f z1^3Lus*4>PktwE2k__Yftgv3 zbVdol**~FKub_{}ff9Fs*yp<;V;~kHG<`p*LktCoQQ;!W?D9-}f0yv>m1zS1ap?sP zSZL;z5Og|bjBgkXd=_2O_r)3`4tm(vtUJzM(P9wxi_WonHW2Kt5!4{gxJ(6R(f?gw z0!XZ2vmb1Y7^g@0gL^T`TmDm_pK2-^oW%eSY+D~L*wBIm{B?}NT|4qqKgu&J0Eg^4z5d*vg}h&QnUxC!8@)Q%afpeRBcI#IW^7 zi$lO(J_68r&3AdZSh?Xv-8#KJm{F+$KL!~0C{B%9y+55RV}ioNpZhJg3qk!Nau(C> zsG@Rae|Ei~!R)L`Em`?1O-jy=fH@B-{2QaF=_omJSw2@xnjd+M1!YZwwD?{zZ6_B< z4;~2_CqCQGC699+Uh^3RsVTPa?3K_RF7U7PX-(sVI!=uG0gmIFpmoes_Kp{aLRldP zO8Qul4_vs{iK!(@1kkY_2>l7@Ex$~WHMb)f$ibuvAtOg`qiaql#a`v_gEfC*Wl`mv zOt{e}CB>s$uvnyh?N+o43kV>7{3m8`AeY|I!blhxzWDy&UBpP4PCsA_STbVlE~0;n zofKIilxtQPnL8xPAFIR(IDWHZ2@s?UuH`l}f8D&m3z|5U-)Cfl)e2xd0GIQv;G^=U z*+oEVPN-iDizIm~?=~hT(SOoS|Bq{qDuZ5q=}#Tr2dYG#M>}soAmvZKY9I#_11ul(|TG{2%1X(F@0 z)KkW{*i^V9Y@flu)C3)eg%zR9CRd)5tM#J=wtM&aVV;(Afn^wov}iZm%LyiQ37 z?4hGsBLKW{6hNv+N=hoJG@|7!DPEH%p!znmkNe8w5S0B5qfW&;v`m7dywW_9Z}=58 zM`Nm2hT@m$A zE_yG5*=pPa8RP`zh`LyC74`fE>$I4;%nl3D{Wd?R1V(YyfSg-Y|H@uEog0l_3?*wH z1nRU#axu+=$iPi&_CdZ zvyGFd=(Dobt9=LMf8nIU-9Z9!fh)RYLl$-1BP@KV5sAze&#o3@BDUuXRaaOERq5f~ zKf#$QA+JlB^>+|{*~8-c!>_$EDW)rIJ2`GO9-W)-Wwg*B8D$-Y^(i+qS~$M!@3j#f z6#+RxN~XUO@#)Xl(5&UxArsBPv`zSJC|Mh-bfYNN0j&uTYzzYnvg>{984*5MaaKTz zXtlC4I6dWU^17Gi74E)TJnHoPF#P2))WN z{uTV#(T3uE!@3(sBsLPQSvbu9ypY%{&vC^&?Mltgp<)J!$lKke4dYH4Q^k)@ynf>4 z6EUns0xKTZrf){2!&3BrC(|pWr|s^P9$`plL&de z@AZQ^?(7j%Ay$WE=bXd`4v)j;{ic9~q)>47IL>OJ!6Qus)auy##{M5u?-*W5)3psx zoQabOC*H9$v2EY}?7VujhV$eEqYIKDw)`R#$bebplGbyJ-J$ zMSVL3zIVcT8+I{Kd3Z7)5x6gvpywh(Jf@h0Y;26KVF_plyOJSwV8OwDt?jImwcY+; z&%K9-%%%XhzsA|V``(KsQeBQ_Fkbk!&q4I)`%UPNZ5fYVpbfFRqH%)Pa3|6|0jy_Bwdf%QGQh3@3#@@a32`CocHW!HYQQh6k5@pB}iE9T=OFlVjGhUPKu=RC1wxldZk3 z4u_gjG-lHv-F9ik5^waSqvVdsHEfgM**q8od4IfmSM4s3xta= zLS4D1w4C7M<8yTvOz?*o@f)QKLlp$7WrFL|Q+=xC`}3>z{^P7Cbswh?LKPm@Q9+I7 zGEf^~F%dYEb_~@93d$B`0i(my$v*`Pk8m4 zO-C8vA<8Uy%i=@E?kcOqI9b6as)ZY*-f>7XasjhntuHDU!w=i}ymfKpsx`{7YBFDe zo&7!fbX_BOH114-S5i-|o5!I1Z``@?0^Ep{qLc@R69tzgwAZUvU8XYOK5t7x8=?{5 zaCfZ5=*@@FK5rv?t}#tIRyfRn&>W(25LXaB?iimF ze{j0*ZttHE4o}zkWf%DEG#GfALMs2%%~3O$(|<(?&uoD+Gv2=4-UglYrP+-;Ze1cz z4L7#Xl#987o{cDl!L9__+|zVN;qcTU?s1XUMSz*(Lw*`|_shKseZ37?j-X`nmi<7* zMIzh*A}S>)i^&Zu2=x|bQd@yAd(Db-FhC=(sWB=#*w=RxlTflti63|@rjNndw!7Ur zaQk(M=7M>(X5^36&&6~Xu zg%TkC{n=l+aqAJ)&x8H_Z4j(urP&(S=kvvsYGf0HwIjdiVG~imy)$TnpKZs;4ndgI1%(u>SDRwpRbnkUE6`kd;)H6&#cv%QCGvA z(Fb!DGS4SUEJ>v7ID44B@}d_GarA#=ii0vc(b*-iru7-B?9sCDDNWIo$?pWV(Y z)H8}hRqaN9*%a~^{4Ol<*h@4t|W z{K>5p`E?=>^=sV$Sva`IEj&1K(DqDntd=r`XeX9tFDR8E9-v6P#gfGqSk*L$5g zGXbF^6wRTm=o<PYxOCx#ocU8PUwl(Hy!RVWD7cZUZA zdqe?RwVv$fQ0^$fz9XW#xqUggbN#cezY;M5#(Af`o*5^xjEK1|0XWPp-(BU7wVHzZ z<$=*MKLyfVtU@^hi`LuND@UkJuNBwnx=d-MLw>%lHRTohwD71K<$J^6hP0we);|#AFxxL|X>1H8g zJ1Ep!=}1W-2SSlYKox{K^96GAh#H^u z$ulg4lc|4VTzSJEAUqW&t*U)eCLU3>z2 zo_d8W|MnHnLfrGsd`~yjXcdi;x#23DZRoXISz&tRhk$qP-tV(Oq?TECjYW1(@AqGk zS=HHeV;VOQ4&+^k2PX~k|9*QP!7OkyzPmrmq!N{I?6tCk)-~0QU-P8Y_zueUM~zvD zcq`*{@}hw}xXrd((X+GiAiLRuNh2fzo-k02Es;o$9)5AH$+FJz2 zniA`D!-1BLIPBNVDs^rIwbo{_MP?xRyHo~LdP9l6kSLWs zk8qahus*PW6r1E3T|G@TJPJ3K-f7BaYdcNeEs>KxAc8t%Z%I z`n?vMvoG)Zb9(@-1Mbil`Yiye6lEJ`gJ$6`sjsjN(xiw54=+#Jd^^-URlmb2X?3JK zV!NKIAdW(rUCO~Is&Ur~xzG=WP5)Q&-K4BQX~IrqrKq2?$ate~L7@NdHq{Ls1~+o? zRsC)t2z@I0@%muEqH8V2Z@&u}5D=ixsGS82h?^sqP1ipdPnu8MddM0fyZezisI$ov zg(A%me-9_ViA45MlB(qEbq<9W!xt#eHD_VuV-3C$7@#mW^W$F?q0k>P@w&YkPPg5D zLbJtUe2*rO`qs{l*#dQ@njPopT0K-Z?W&a-N})SGpC!e>RG5AOPPnD7xWup9eUN4L zbh@4b6>VW`=leQih?>Z2@lN|Fa^O&R7zRim`8@NIYjp@$-fcfl;rJNU@A9s$S^gcXh%>p;INP8g+YY|N zcjRJtN{V zj+x-lSE`n{=SZgY{1{3vxi?5IR@}39&)D|w7!H>}A^pP4TkKURU+>Tu&Tlv`GM?0L zQIEh3vC~%B3J~d+T5j322<*3qff6&0X|gY)=eXp*EmWq0SK9#nWMTG%b4I4PJpZmh zPh&t|&rVDX0#N}o`F(W((qzz7Ha7DUR`dlVv|5%8q|GHRApdfT}{K4GQTCp3D`K?zX zA=cTp2iT1i7@w4!koVIDtLODxEvQ3f{Ts@z>#IkorM!RCn^x%K2NY_ zIWRFzL8qMhhIvp0;jRzc>nN)H&SD<1Z_aXVgo3J6GL7!N?^gUcz4udf{5W&i@ZB%l zcAQ(U)Bt*^v zQiu}u)FCCjngXQLThZ0CaD_7X0_08tgJe@VeBkQO4S#)D3QCBOEmr7Y=&I?^BCoSu zh%{?N(erW(Sl$b{?a{kM@rCc~T|c{;UVrGB&VIzDcjnuQ#vK|b0E&RixgBe3{bD7_ zcoqUS0u}_@c?`>|^{%vk9^D~Qn8B`aYnK`46Dy$KkGtFMwP-#=E0N(iIUNWvHmp*T zlCt;nJESE$%)mhvD5PK(NPG6t!Xc4|i$fUWIX+jq%W@MBRcgm>jl+m)mpl?2%Qk7T zwkw7^!KorwW;%ScHBcoqN-XQ1URM$#8V}lVX71NzCaL|c^5m&gR4t^}S~aPcw>K&t zUL?pIRYH|AN{k@t_`&1;e071-uWoO#OewhMPhcDkTNM^Iw%f;3+6#w!FiE|Kk@E97 z`=sV7HX-k7^XwRI+bUyPr9=})1fmamt!5=-xCwH-99>#y4KTxwzLHNjg8KGN{$y#< z4&83=@Js8&J)C9EbUR@%ItIeiNFuFVIvxj?N_{4|cfG(rQl}S259`8pma^scn?3+_ z*LuRA;csmIf_6uNsnq+&%n2Wi|5Gsv@mWu^sc;&~RupiD1ax>r7<~(_F zKEJZOo?`jQd$wN%8Z@n<%k3;MU?hf1!h1VBYq4S9o9OVuIE>FOMir(7QQ}@lO0C62 z(D5CgQLYiY8u;hm%H|x>>~9C@^M)L(RsdYsHS|n7L_K?xTgQUyj`|W}ipKElV7Y{a&k8nsb;#prZ`kxP&r)}kO z2-)&CNpF%0GF<1(gPKlO!_hGDy5Fa2>b%Z)x9Lmb$`2@mqJ+c#7rr37X`!u06*sai z+EnwcXLQtjc7Ne5h=Ndvlf1>T2vOsa{!u0L*=}5G_HWEs3-0-jry~fE%^@pA%#cqU z+YL!Yqe?icWKnRBf?2tf6x3a@qZ0ART`z`nr7H(HWE@utyG6xDl}crz7Y>PTVn)u4M;Cb=8`;*ia;5gngQS{D(=Pm4jJ${N z$Qf*a9A_P0CTdX7kzdXs*xmA&ah^=trXSRuhe?krZxY{;P3>;&ZLO#xt0g!-*I4pw zkemK1%i$=T$P!pK@#>VDT(+j(1ALb+59YtFunY&k*105STESYeDQ^R8>=y$`-=nDv zwc@LIsUm7evP~>KFJjpp=kt{tYSX*jjlbfSei_q zPT1AEAKCGy?8IaK9vZ1$txYRIneiZ8&(RdBE0_rW_4?NRe2sWt85=W&>od~wCq;kr zP^VV;B%qF&Pb3AdQ>oQmNFeK~B4#!$8{IsEK%j&E_E=?2JgX|Y+0!FVkz_a41l-}y zbjrH`F&y;nWAd{M6T347H@Xw=2Pn>@Idb@+6tMjY6kQ>_B{gKVzj>;Ygcs&wWN_J!PflqdCEca1xK_27W*CiN zBi3J~%)dmQ5A64%=eu!7r2RX4f9*Z9McE^xYA*tx#tuia@}F?RYrZ2V%+J>?g?SE`){`nHY=(YIv2FWP89lpDlL{FKzzPU$eRo zt=!UPd{zr*ou3fJP%nVGeqbU>O@ty5 zpKN>Xlzo&ud4wF~Box%=8XQcd9P7EK&IRYPg^Y3=c_M?N7*!1pW`X;3>aenCtT#_Z z>aAZY%*2EUq=3kKAgMu7Kd@n(kNq!qQ7 zP&wA5!L0D&KKacYj4=`YN<@(;#x)k53BkTMHPru#{u=>%##uZsZdQS#rhW4{fXU(8 z3JAioeJWrI5R(9%?k5K2rBvC~|BS9Jqh#{Zq@)iXWT4biIjs|^rEc7pmD3QpkzD0=yhy2YZe%CVUNEAcI9VLubO zK>@$CVxBMlFT}(CT|(*S)=y}!Gk7a)@Wp&n#|nXiJa*15!Ma|a;cKdj(-?ypZz5pi*&m0E+aurR-p64N}D@L*Pbnkx`JpSEi7Y`ukn0E9L? zL&i%QcL+^{b9H^z9muTi?9&el5Q0j|!0Oy^)A_fIvXuyCII}nxzu}t!*7sz+ZJ@~4 zLrXCvn)NcWxbvDWIZ{J(pb?rVVdTE3S@wg93IK#M4ezu9vv3Jm$#sxqXN57ZU1I?I z@d_3mdi=WR|Kx`5_Gt;g$N%FMm}(O7B4D0X;NA4o1({44PVrP&;@6W65^+f4TCr8w z3=fjQPpV5>*KaAM2z-ULKH1Vi(V9z>Hv%f{M?$ao0#YvDsK$6RBsJm;f1%k&6gm17 z%@8Z*a`|p_hH_y5MeAFIf~fZF0l|eFXDY-+js9Vv%V8lW04{~MPSUz0&1cBaP#4q3 z1xbz9Qwn>-_IdxMHP797ECqHkp?=y*lE=vSam0o#qz4PsGYkrT5O<0g-z~lgL`?p* z`IjzHk9hAl`u-}1AKT)d5b070C8n3}QS$2ci7H`Vdlw~gE{0~G|N4;-)@x#cbzg1N zj~>4SzDj*~Cj9Wki!Wzf>B-HN0=p|3|B#Pd)*WUxNt(u9;0TQhrqPrGYtm_j({7^3 z@NmiGPAEgVx*WTPJ;s%0jHiT(ElK*8H?B+$c*Huv2|*^mWye&P-Pe{bJUPzkO>@fS ze^BEs13oHnuM`zLiVqsx{t#&nN9mFlSr@!`I*Sr$%D#UGRiutHF&4AA=NlHke?wX! zZmx#gd6z10c|?=Miv9u)riDrQ0`y42c?fbn{z=}tbRcOoDsbOFki2kuOX^^9EpM$I z<9z-T&aydFiMy<+fy^4|;l@v8DywBks!XS6DlX>Yagg}L*AOMw!h2$HSER&Hbl5%< zq`SuWY2r@?i3AUy@SZaqi9-UioLh1Ri8RFuVGIrr1IWq698i2;4pM5;cvAQHVg0#g zBkgjTqZE_MdXN72lY?;nh1V4-fLnA(PZd|nm^dMP36Vl1sudN)<6uBhsU=yc-`Am* z{xOSUv!ldGrv!t^FCjxfFk>U4P6q5qy{jb7mqgFZHxa+wmK4om-4NUOMzdb%xL5B! zk%EVPY}h}N{v+}ATEOv2)}#_pfh4O%qAogk1&|p12Rc5?UV;YcSff02?&E~hy6{;1 zK0YavflBnE3K9$5_-ysp&nAAsC+`EQFnmJ)*#L>rB<$@bcE6?(|EWS5eA3KB8mE{$ zrwnqG`?oNJ)g8OY07J?#`WMc8=*AqJ9D*u_1KYvX1?A~-0WfPzi7UOZM+q;$$CIRIR6ii}4 z`to%bzVD(;fYnEHA$}$koRQ&)Hu?UFA7%VwRtLR};ZC%1!5$)+LZwr(Wb+=WVbhPr ze*E}Hk~6@mZ}4H~QLm|MGn8R4M?z*IZ_B>oojnZbAIF~5OH~x8^*YMzZ6evJ{-5ve zM7-gAjl7ZBtP!KLht9Y6TdZRVI^!s!>SE<)H{i$I<8XgI^>-bzw67}@I zC~hHOkR&iElV>hLYBmA`A&5mzAh!iZAZafz;nb#OmXW^gfuB9BMg9RHm;VkMO%x@UEaP4Qzpp|JR~C99YU1!JP}2Yoji&t9q>fw^La*a8ZOBH+U6C98KIhAx zics*^$bCBqK4!!kkSH(sqL7pqa%so)TCE9qyvLvB_72#Ac;yXlfCboukDV0=bQFb~ zg4b)Ol;^npGh-%&O%O}|CA&wyT7r#CZ(Ht&(l&;t&ixnenEnAw@0G};vWOx%kNAES z?${f9J4{9j3HZ~bwCKKH@cgw&FzXnf6JKO+y}5x$GL3R79G}DL)OlL}{vDH5{fz3c zi3jF#aj8$spwJCPTmev|dBcmy7!6fh(zt6C;UzOvYyqbJfQ5hTgx?85DKT=EQT#G) zAZ$Yc*IBgjE5MlDn>uL=kn-lDYRUA{LKfe0WxL9m*_AO_vlMlDxg)+`f4N4c<<5C2 zH?c5Zlhqdc-lh%ol11RmIA0Cd<`ga zN*hIUZoD8Yy3m{8@^|h;5--Ek0c!T**{nV?sTt2X*QD-KK_nlNKA=!>1CWH@!1#ND z`jZqq1j&VolQMmXfoMbMq#3(r7~rBg)dG}B&cn3&m{do5B_;>@d3O>m`Rxry9Kw)x zlPQIZ>d#Q^%V{LNgM7EH%^tPeE|tWSnupLlY?#<$kmmOZz4JpNlwpYvgy4&Xt_N{o zBI#iqCO-tdyaFK^j1HXB&Ce=KVp_begZahrZCnDXGn^-iDtAZVd zCZ>u>U1DOjnH~QeiI==tU}i76UFjw>6fbO0f?v`5iF7~=P*fw*`eM30;ohtfE0xhy z*MUrls4t$?wO*~Ch(pWVxBRRIdR;=%!lzmpr&J>!I8;j8J;X-B=H7tliJ7F5 zzp1`N@W03K+BLMgb#@bFou6q@8|?S|tJeKyj^;W91CzyM{jg5FE5u2^EImpJ3SKNa z-qLta(F(Jl>jd(ZJeWeTN)ix-iJ@F2$6q7|_HPPyNHGU#4l=4M!ZOJIEFnE8n zKW9dPCQPZ~sR~1>EK%rIv(By8bitn46fizk(SM#05OK_P(XtQ_8l!?KepZX$#TChV zs^xT|kBeFs2?3YbnVcO%8lojK9me3S)$;~O++m{5Sx~xyPdcFgLpK~*M&Xep7CGwQ zk@Uzsus0e%1kYM12CFs0iP}I0?^$p}$ghtsWV0gg|J#g76`xmFx$@=<0}UoajD@np z2eJtgPQQ~Ye}A9h^Y{W$lc)qx^tLnXI#K?Xis zg_ZadfJ0MSdGR{dt^Zxb!Zoh`LZYWfQ{th*|5?BUcFt)!oP_q5iGq^_KzyEs`mTht zJt!RswE^=UM{ZD9A$ScAbqodut!})N7ph8rZ2I<6B$3xEOqtP=ghW<^&p{&kPid#) z8VXLyqw70b66hyC8g^Mhq1rN~fvcgmw!UF%l47hhKC5X#O{n#n<+>jmaDjNDLu6Q7 zz}EC8#}&PH{m6@g7KRc!|5QZ^gV@xjzLxUW|MWn#e8gKoC%z&iY?OxNL`NT<+(tik zdK6XbV=O=e=(FBtIJZl?XMIZ5{Ax=vb|7L4h`T>oUcwm`Agsoi*KYG4O zD51w;VNiYa zs)~602G@&^H=JSQG*dE>(gN|&r=#z#%r^rPgBsLY59>-MT4I`;hbdV6B@vRT_ zyHkzy{sg#azmNW@cw^3}xUAyuDR9ZYxewtUl3me)mt1?ND7NWVfs$h%rzlbY_ehs5 zU5B@16i_Os8Ze#5oYbltUYClCU z&r^=VK^4Z$Z`4>7JPrx)+fwq;fFi6=0hf}_xnESC1p`W+A}f(Y*Lad25lc;dlk~73?5WuW~`#qz1FZe-|@NcIy_l zZ$W~SSz_uDH~J-BqtHRRYh2{bUkUc_MfVYrOTyZc?!!jb+Ld4_kALhC3sl$;6lo2k z<(q|IJeaq4^46kG;=DdQ4G*5G(~Rie`d)iBxZwFK5t<4jthYJWm4h_slr%c~fL;HN z+%y%r8vaTJ_T}XzJf9L8i$c=mBZW^S9$V0`TNXsQd$TD}C^L2j+#2UWkLO1+-zFxe zhBaTTZ?Phe2PmvrvF#roN>M2_dJ(*KgG<*=uGPlG%zS<5N*d{fZjChLRbx}f*cCa^ zK;71xvq2dyj!9Rox?s+P4>_Qv8gbk~5JRm8p}&1s)WuRX`Sb10WSqnVr?*&v8r<`7 ze1V_VC9b1;g;F!o2qcRVZ+Rj;9+IfWBz@IS3wsj#UZFl{_WS!?_&0#0hV>R%&PXMK z4Ym8K_rtWZFNfOx3?q%)ZF?5B@Ki^2}_h!g<`M{NgPYW zTXbV+UNs#>ZJWut=P(}LfycU{(6TlGi_%r@aYw;I1xUYxjVv=ThiD{-c92icD4(1 zyOa`^1~bpS!`svDDpFHqxmEp?B$Api<4>TAM{y=zn{{ogK2o`~Kaz#aOokW}hbXDo zg=dp<;M$8uyXgK<5ZXV%I&00m`7|Xph73~!xLaBWDa77Fev7I%1iOWx!`feGBFr}0O4`s}|f&y1cb_TWaiIb^u`-+ObvhIW#J-?fn z2fM4aCOgO104RALel%TFg9EJ!^+^h`>C`Q}jArBZA|7cX&m)EsqC7p{AbJgtna7r0 z*+IgNIV0QsR&MvIGki~-pFSkr!0PQZueIpFg36W|;+pikc}!IBzb?x7cTuCuk? z>n@yC!nrwKlyBJK_a>-mYssXp(?0F_mguGwHX`8XEU#N`ntam}IR5Jc9TI)a^GU*g zSFspqDt7hx)Jno(Ia9aLsJIK5M~Ux795cU7&$%G8yW@f?{7B<)eOox_!q%-1ZjuKY z&Irn4FsMxf+{Et0o*ZhLb>dHOxwqM|Ci@!i8hA|KukY!}C`{#Nl}a)n^5f19q6^*g z>f7S1_6owwIE-JIzwpR?A;y|o)z#au+YApL@93$SNC>KLna&a_-p>V!1e<@KZvfsz z%^{@Fy0x0~vG-jRS6H&Z=pKf={=9?xxBhXMW_vPRXC0b$yrZ=+ip21+ho6So9?*pBFh16Gm&r8|Jh@q@BTs zSL%JQU2!U*(FPmwCA@q%C0?I`{dxy?Z(|8XNcfIZ7|xTyLRXf{@)KLL(a(7VsT8C_ zI0Ni%c*{ zDB~WZ?ihAXsSTNlnxkdSZElV-FpvvhE^pY-ZJ}Ze4L!+}+!Y%gO>VQ7ECttp0U!{$#Zv^#*PB z{T%v44z3r22Up2`G<1Pcrfu@-sB6ZnBv{gQnR+0@`?us;?^*#LW6rG4F!SUmZvo-t89MO*zeBsc9;%bzFpN0vLII;O7?K3H<5OG8cJ-Y=hrx?Vf_iOJ_Szf2ybB(LHRTIcWyKiVr=|sgPcq^;o z0$ZEr(u&P8ZzjQ#j9>r*@SRSxsofLW*tO4sJNf(xL_ii15&}P6EF&rUeRFpw4B}(Q zCnOYgx<*RW1@-buj=_SfwY$Q7?OSk{b!k+M>eb=3g(7b)jz@bo7vwctm8v3M#vqcr zXf(4<`<#+8B{PJwG7D!LzN#UxyObh`xjkLLs3ee^5Q=7x@hnZDEHYeu7VAc=ma0^yHPl{bta#$tsM2P9a}yt!f%C49vDc+VXzN63JBe zNT$f~LIa7cg?4x<_>f~-zk|?U>cH{9IND@%xBbyL_I=0XpAA2HY`FX?y?%%Yf>}%8 z-=F?POmYKU`9|^l$F<(5X|)x?Xf3{w`eNXXa_t)It$ugGBfA+P^ws|r_4xX96*EHM ziJ)7Q5*mmof)LMI?r<<(s?|b|Nv};A$K&$eWao3><<+^XcK&BZ)J$2H=m31W$hx;% z$Md?M+cv)r%XQ=EKiJrIYl8eQ(}Q|i;E<%sjxA<#t};sf?wG&MTcT9<)^zY6EQVbW z)JhzLpv(4tTt%bOz@9UX-jf-gESxAlUkTO7O2x1Mp*Bi&I@nG@&-^%aC%3yEm0`z1 z0-lf}BlbstM89Eo@id0d5gK?IldEJy@ zvdjrw6J~nFJUU0_lYzAaR8{e)|B{UnAA=JE*ND_#6peNWuGl8=q;mCjv@&f~uK3Hc zU2_m`2y}c>-8wZ;pvw&aXyii^7`fyR6Oq8`HUWFR)$5JF@&AJ>EF~ zUUq2-GT1OLsK=Co%}0C&5}#TM?w-FNYyHLb#FXieN#RlGsfWw()d&sp;w^vj0ylbP zAFf-1uQgkuC&dk%yBfo~F$B?UNa`KPenb4rXn;!6Tpmn<@YO$>V=1*{9ch^14=las zeco;;3Ioy$7{3X+IS>9YLm{mpcQPh77&$$7{ylWXp{F0{In?fUoHpmyXhI5-!-)3% zB0dy5njZo@eLctT$4m4U@%{kiblC3kl9?ZaA;)!KvLjoPq z9SJ;1V)hdWihXpzp}2$u64YD}hLD7owsMf~kYzgT$DKFm?N<7}JGgyz_6NlN-=57A zo`>s#D^mTCkbldI$F;lOw@tm%!MHEgLrkJk3o4-pw*|hGUe{K>Tj6oK;~*buz_%Qv zIoh2o!=;-XEHK#aYex>v1(vnODiTIzI18yeAVga|Bgnmz_g6_7H6G7Ftd_(W+==h) zlwKo}H&+FDIr7otz5ftrqR$iT@=*R~mC19>e#rMlk;s=<`nQqQL#-!LM@wxtkOsi_ zV$AU#jPU*r@*LB~{Ue6_C}c}WM~}Z5JfoWpJJg?sJey?UdP>D$BuGmt9$dy zY79-}ot=(X5Ssce^CX|DOMeh?577__RH5!`D_3(FtuKO$fEZ3iE@6z2%Y)ZAR|Q(% zmh?7kv48mx+&a>1hnF#5o7F-eZc5*rHgc*rUFC~pE9)=(<}>3z7L_@lF?>qBISLa0 z<&uH9`4-h5nTL<4x%EiEY!+3xTwH%~7C7HU=c&58s<2nbO8h+r z#ocgNMnyn{=Y3oR!JI5%8c)U5jkgO@@~ydd@y6g7XfqtwNgJj@AeO^;i8J#baP|f{ zfg(jO!P4T`qg(DTx#BdBD^EYi?d@$LBP1#+s#-8UEG(=5NS2zcVj+{$kraFhgi%Br zVYI~13LY^OOoEAd-MkShYByVaf!*B$8vP|aV|-dqD(y7I3hDWTZdYxu>$RC28C#(H z<_wWJ$XSJ89p118Y5cQ`=SP}wcswouwE=V5@pB9@M zt`-Q(Hd|nU>>-#J7y>GVFvlxwUw%gx>;~bSg0a|B{F!5y0LhZ%6#h9%AZ9 zfnaB6Z#0_|0%#WX6@MsGFR7@~V5KIsEE$FlGh8Rd0fC!|< z`l%AfsyI?VJ^(zL=bwP0&A#JQ{c`?hz6NKeSA(X=Ja4~RDHJB}q!S05z7(0Stm(n; zbh-7tfnQo=LyxT4-@-TN$~fGh=M86fpEz%5UZWg!5eqo+O0DbDq-TJ*9WAnLhz6y7CFxS z?tsV67ovXk;C9oBYaR0(m^6l&wYkTyw zOCZkS>fpYsuHkZg)w2#qCgx}TA9X;S1nC#Ocn~#5@p!r5->gCe;>`H?c=ZmKDhmxX zq0!K5=e0VcP_B4j5FjivlE+XFb=Gkx0s{o;L&L?@I=MM3VH=cF8;?}m%F}v&XpBZ2 z-mk5*?UOS;2MTZbAOJ{>oBLvj-W+_gd;G`^vaToAr)u9PxyJ_hWIarOA(E&wv1(NSra5k(Z+h{ zvZE>DqbbrCXQ7zzY2oI8f6Iv6bkNdCFq`D%0W z#%;fYzwhX3vUpK{fw3m&q}dopj&b_MkCi_3Z5Av~9JNH9Kq(^##qL1|h4 zvyd!jV}M#6Ax7_rPE`dH|9rq~bdBR%Iw~u4Z(48!Hu|DrTL=FchZD-kt@V!bdBfCo? zoYJ$iy;YgM?F^@%i;ki|LfFm_?Y@|H8#T?>rowlz!Y{iUck;ss+!$9|gM*!C%|9J_|7_2t{(L=SP%#U)>BiD^^M^+KIoh&Faqif1eX{3jw^#i8{r!~$!R)8v zh4;aR3tG6y?bT9CiO0GoXN!(|+pK;`!~$xmcti;<4vIleYb;K>xlaIp|%DXAWAkK5}T=rT!r)?!1cse&NK098pBl zaS;SHo>!T$=`tdw9h#a>#>=a7G5(`2doZj$F%kX!{e54{{b%gv|No&^tz9eW+sBjD zGVxYgCJ#8R+?lW=ZxPGEQJ)n#qSZ#+wH+RJ>^K&o#Z3Gq$k4(I>Fw(ew!_b_*2Q57 z^K2VSh>?nxu{!I@11(LirNjS=QPOx}Wp0VXx>z3|+L*zh6PJo?8OeW3jI)Jj%Vya& zWkY-;%YU;BIvG!xMXl<$EAzj` zFX#IaO?N=th)E^I|7$UcE^{ikbf@Ouo%W1Z>iU1Z%4FR=QI?qkfc*OY@4VAUx7$pi zia20RN*T@n^=2QmY*8yJ$fzKw5&d6-2w+GFs!lm6^){HM)XIj3^Az~sFrtn5CjGe7 zx@(QO-2D-~XA>IvKpGboB4%JfTvSx#)}@<91d1#EaIpf;oh{ZE-5ZXPbyaK^1Zd%; zw=BgQQd(KQGsTyyr0T!sF>n!7tBGtpGDxDm?xqJK~sK_K)v#@t{MDary5Vt7V3G)zvDI4`f zC8zS{8Dp=$#RqJGFIfnmGvjgGEn|{SG}D-%(jbi|p`xNvpaXU<$AIQ*k#(n#x6~%u z)SZDLpfQOOu=OckbxS~3>9y{bm}Hps+`3UBUV!I%tlb1*avI&pg~L++22>P>N1{pv zP40f9;*USeJ=ED=s>Kn$F;uztij4n@BF-+i=~@Q!Bta$`&CF;M|8iPw|Njd_&Ak9{+lMF`6B75CX(y zdMo1`l>sn1?tuq9ZlKsYG`lB%pY(8E41ZN{3R_o*=xT;3$2btT$IMv1=*Qkle4hQr zqN|Y&ivI>>9u?i`r0+%LkV->#Q1j;6fHMDY^Ja``Q=!G&XkiK!NF5)k1&{aSzLBCG zsb6P2&3LtYiit}461;1Vmq4k=?-fgu)~sSU?;oQXPt2}*&dhT^zIGp^J@g%tR8K#t z-l=4L4oQ+ZFOXPW{(vU2NkYSHUl9VE7R9XnHQoAkyqe8X3k99yu z=wzjiWGaJw+_RZ|}AWzQj#)Cw3N z0gNS0-4rfZpl9t4AJ|d5H_McnJwV@UG62?3A3Rv3k{`dS2Q*xm3j|IwgPV&iyNEM!nGmD|)W-6OA96Vdf{B5Un+edmHDKzECgT~w1xWQAU2hOBfS1shDI8=41oDDflULLP z=uw9kMk}1PUjP@)-zm2!aNZ;VW%K3rHk(|!?iVUciW!-iG8M^H$6GBq>7nGlTpC%s z2me_o>8!i3NJz+cT#qwlBWXP%W%~b)e|;iCfEi+I;c`t_$L<|9#hf4687kdBSB#1a z=;8l8`K4>n!yefxK=LTN&}uqJ?a=L&K*JrfBlDJp{O=z~T_|qR5;MGqK0%{~UvPLB z@87E#-rP@i8osta9eSTh@v)FXGP<|E9vRiXSGz4w1bM!RDUoe@*UKU1gW}Kz4kNh> z+*Im!Bn9F!`r_(F?Zs@l9HzWAd`|EQPo3lOH(k+bd}0aVF%;f187|gIPUd4EU41zP zdsIQqzAwvTY9Ea!OUg@0Q&acC!N2#gw`Q-(ao)};PIb=tn|C&i4_;Wj#@9M6?PaN2i zSb4F&Qg8o5>*%j^lQrGlv34qwKy{?OKIgq^ojJk7(pllvgQm?utOrh~bRlZHdt z;;K!7ItPNT1^*5~m|?1;^p0)I5EeKJ-vzYBqv+kfr|zRa3_S*Gi^#_)#=Z9gGw1mh z2`y%cNh?;B8Vw;}M@U3OjD9yQF-JuydWR7cBl7afqr+1f$BS(V)ah(iD}FmHGOtvi z4REB1N2P`PQk~?o&@W!1e0A2YV@we;njtwIB(AyEXimx|weSB%sDJ)oq)}%GHIYn5 z!pz(dui$zhv&2P0{`6bd_nps2yE`;CO#c0>i19Vt728yMAi4iTJ<59JZGlwx$?W5vZR{5=u@GJu4;i)3oao;YIl>S`{E)7~#H;R)WKg27 zJUQul_8P8UGH}F1#accUq5Q5{32Ugg!iLmDfY01<>o7bD=GaILGMEOpYCtO0hp=bn znV@DMP|XOy(f&QqFn`vvvpm9{6F-q7DB=gn)&^tUkc`0aR3YQezxjIUplSIqOmp8|3uLGq|&5nfakSa+g(}qb+kuKOQ z&0}Sibl5!XpJ-CLx&$_+8frz)H1j6^`H$F-;13QC)>wZ)N|H;)W`4UeLu#}yXUnHvWQ(P{z$-T!PG~;B_%?O`ZvixCI0K< zr0;l%heoW;-42&lB;6q4&1`4K9xpO)ANs3Q;NE9k7wa?TDTICG#J(&*t^iNk^n3sA zlo|i(tzrS}vGRE7UYWb%eoUm!57QFO)vuD0iyMsDCt6P%Q%O;G2aXHez z%HTL-DsAtUwE`D0IlR7jDxuww=bd-LSE3+jg49 zPGe4N+s?$cZQFRK@9#e!=Q=ZU&6zpRdDhx%?^_}=tmtE>O5O9Pu!u3Kg27i~n`L2t zMmBsW7%~QWllU!+^RTcelhuUH<%=5%X6A5nKL(zrLM6qM3U5zzHE1n_1SG=6$VMAQ zDCn6ww;pMXLyWDNBLspDOOfAGr&VnL57>hm<6)GwaD6mG#Za;kN4qlBO5=&M<;d12 z6t?;RXN+W!Bw$a}1bBB@;UxWpArISo3iT%Q_OQkB3V3;+og^=l-4SAi764RNGlJ~c z!IV*YwcDM)JKX_`>RG3{J}IJ)e1jQ~Vh$S=_{Yn<;lMi6no%3mt>3zLR(RRg*!^^i zh-R;yk5~ewi182)6#9LVsFn8o&aFvs51iU9;aIU z&Au(1vcXg-e@!-aP*5-jH>!?SRT+3!qE|)Ww&6g*(0NCEi%Fj?Q^uZOn13_Bs8?h7 z3^SnX$EVZks**WM_HPsMYtp_LY`qUrz`N;&qq=?bw;XF?41*Kp#8{6>XN+CuQ728; zn1LG-3X0+>HQhD^kr;E>h?Jrm(sVJfjSBF7Tv%Kzes0Nv1E;d;A_AB`JzB~kp;J@} ziwsv3#sD><#itLf4{ZqsL=Q!vKQ{EU8jX3ZhgO?Yy0+BN5R(>4PzMY~9s<&<3QD>% z728m|`swd%)>C>>IKCE)TwX!7>N&EY(jUpx4@WdGV(N&%@U1kE%O9$KGvbc8u?Huc z$y55(YmYLWN4Spj-AHL-*VP?5MtbKB_u^oN{~YRSspbI9K;l?1WC8kUoxUB1@7jhN z)gQfU%iVBSspX^-T6;xW;&q5ok>`$dCA}(?l3~0RG1o*Jsukvu>Vk)uHv{Bq#3v;U zfQkOJS#Jcba*UO27u;;)fkBcWbs@xR@{u~f{&1xx)6l3C#hYlOwXL38j@IK1KS&uN z_k@wSVB31bF`*=}t+K4Q`FN8=6|HGF#Q&i5$$Yp&eT?MClBt>5XJD0bJ_9X^5Snc{H3O$dC|KGBu!X zSeYx{=`AJ2%eNLK3kjJ9t)RCo#lA`>E>yWA6|-e}z?KXMKU(tr-?EVGZM#Qt(~GU~ zPOB26X84jd3N~f2YEF29%EPWIHd@!}y_T1XtP(&mCmD!T<%Vt@^}^KN{yu%>O?{PFfVfV&lP0B&w_{E}`&0hk#ze zCgd7bhv7d#?q~@dO+9bqv5g(u{oZ?5QHAB89ZVBDcN|%<2g%eB3WxGnO|aT%qe#9Xu)pfUhuBSyfS>_4R*E|ynv^XSm^N{&d+ z5oVuw8hyP;k>EYrZn@`BL7dTUxvRozD3|VDS_;NFlB>Mk23i0<>Hp5Ti10%pK=WHN zOM)DQlCs_t=QF4Fb5RZ2gF#-Kon!ZHDC#=t#7`aFpAw`l_|q#)>Er6O7^zqw(JGtd z53;sStkoQFi85r9-Af&9+jhfoD^Fj1LqlKq(dAe;!P}dug*UTSlMHy4!z#2dN9__2l_`Lp=W- ztq4kJ{^!n!D6(8DGs&CEaOx0EgRG=~pJ!R&URt7m6V)950XjvXXXfxgE@LSO4)xu@;Hq+`6Vp?g zR-WJM%1xIokDpciXUODFRQr%?JHel^3mJ17n`AGhZUL*Vqt8Uf$DlN?5CcP1^`Un* z*#zAa{c!S^^H$_e=7>SFy#IY1$5rp?Vvj6~wj*r&C-NFAjN?O3W}dxibB{*p8-4%2 zYxf6VtnY7OrTG<`*sPer1H}*19s-l~hR0o9Zgjaa%65VpI|ADf$fi{UFE{F)@)VN7 z)`tDw%5(Sd%n|mH>l>IWPRosBH~n!-4BI6JLJKyo-W%mqlM-qxrRV)bm-XZNng9Ov zpC)L3Yn4hq;r>p25U-u%p1bh z;InL3l$duY#MXREyBAxe!r^~IdF->xkr`4DSrT!+zDLhVgg^c84`n~9^!yS`J-YA? z&(H;)?v^cc0_G6@a}_=8;b8R&fC}nuoQa;N(b(GzCMc=7saa6w&4im-`S+W~ajOMA z978yn_OKXDR~&!UfmHH7?RW~C-{5_6CX#-KSb+=l#orot(+Be3)r>rfn7n6X>Bq<* zi{cxo<5}xK2z*9fJRD}{V-@;97D91ef|+ksm8XMi;AE^5^j=MXKN<6Dd(u6f2dEX% ztWqyJUCXd_q6TCk`^5{s+m0^v#*tYKJDS%7&gJ<$E^w+&n*7oyX{{IUJ)1C{{&SCo zQ(tIf|J9E{Qzi+O*54uj;Yhq)A-pJFUWr*tf;29Vijs1x+yC3rxoav`QQnaXPOYGz z;J@B4_Vmf8dnkdWxHuGTdQ4GKQP3!H(EO$<-8^-APx?qP5+D$G!dw$IY^->hS6{EJ zL%x9>0sJ#d)mpfA;#PBJ9KC&cDQ94?Eus7C$3gmL@Ydp1klB`t=#%vO-`|M1J>N!d zD*eNYkteeWc&}*z{^Qf+G$T6)90u2f?z<#Y;1Gf#R#(!1-{jO-H&1hvjSyd;*Y3d+Ce7m+G^ksiyA8Y5tWX7wOrP%}j`~gv4`e|a1^LQzH$STu z%T9rt+%$mv)l|?bC2ng=@AY`0LgO0H^8UZ>a|MgJ6LO)|Fu(9) zd9YY(!%TL7`@L7LJ*i6;`bK3flu?TNg~I%@ViZiC7>TZ;^uo+_cuLm5!Ex?i{(%wC zX)elv-E_mV^oK&eWuO~Fzm~F%(qr%(tgO~oACXrg$DD1%18l7nH|Z%W+>`YW* zhgY571yjQMjQtR)OWMlb313hws3xdH(l5D1aKcT5l%D}Xa%<2`G)Y-(Aft4%uvBbO zWq#-1wKRfcB4KJuet$C0qZ~TIygR4m0p82qD>y1FB-D%DMqnUw1^t(V8W4bP^{hX_ zzqvdDMyqBTR*XJ4r1zENLPc59$~vnn+HY~2v4Zao8|?V?TC0IDa>reg?}Yydmlm&Y zelQ;j|Gek1YtBnrDUdp$pFGWR)gs|twDw{>W|4@X)zJoF+~M-_>Vn-d0;KxDOG2(~ zw(02ffo}pD=~X1ubGc9s$(GQwA!4yB>XQd9PDE|LixK#kxFR&(9=%l$D0``Bzi8#a z+Js(&I@8LTEN*KZ(7*;xVW{#Mq?V4umn&I%aQC)oCxwAlW+5QGW+JG5ZWg5RVJ)UC z{`KqEQgyy~PP}MM5OkdC4=z<$2PDQtkDnpU5#INfRRcfv=3EEzV14x>s4so8PcI~* z&hC`7sU7kLYgEvmRpM=Vd7Fr*oN%OGR<7@H@`}XW`}VqjXd9tBms5@ zweAote*XF7KKNt|djI%tU6+15-}u#cMS7?Aygcm{)~g<9Y-*Cnc37H>f{OXeyEyp` zf~x<>WrXTmScLQSu8A5w9c~pl;`N-I*aOtN{3=^uCpU*eeC5BpIF~Mvz6AG} zHY4csww{}C!}{Bz-EU%1_M@?IbIIPs6?J4J;+bY*-*BTXd{N$cC2-ll#4ws}-CBCc z&7$A7Z?W|y__eY1TqvA7zO0Z@=F(u8zU7`RLgl@1_HEZuNdm3=jhVJCyr^2`a06tZ z;6;_!eKQ?>_wjNHcU?cQ{-9zzOnqe9NWl&#U7Bon9+N=MW)Z}272?EJ*VS6S;PU4FMQM=XctCRM%@_Mt} z{-Rpr|L-6zF?*Xtyuo{k-AgO}^lE5xQhpwOcQXHWH2d?tw3QscTzmeKAmpk)c#qwk z2S|v*cCy*+eng;`^wDWVR*>Z5ox#z(<|&$Tf8^49^YKFCzK<_%gRuPiLyAkS3p}HO zxYXqN8n)~g?7ovRStSW&E75o!&1#>xx^;jfjE>Y#W1lpttecGRR544h#Sf}F-rUARjwRM>?YGcgmZ zk>7mkN2}YV??|h(kdV-c_PYJt`b&tgc0z6L-CP#!RH<>lwR#(ZcDVfsJKkkqOfDjS z+pKnbOz+`wIfh)3kUHp|h}xYtVCS@uYquWb3y+XimD%$4{<=9;h^VvsHo4Y72Z-U} z#P*RKlfW-@K2$s?Ul;y6pA$`qQux`N@ZfAS|2v*(iAUADH}DXX=_Nn{E|lA!#6w#h zR%%Ke#Gg7^YT7mvj>NAITdU8O=NLl8^+HC>52; z9)K7ooMfuSH`hTY#5iErYj51qTCA2PUL!VCeFM9y{AlRg_gL2u=}!#cUyv`QY#yV^ zRd0p3{iI0Ee(%We-aK{O`90VmJe6;et$JJlLsDAFcE_}pbPKtqQxP`>yYxvacWB$c zN2Bs**sRBRKYg&W7oL;Qu|(9*vowGDL%RD;Z;%{bmJ$tQe~~sNq0lxvwA*S+&N49Y z)i~wgZsCY0>;(ANn7Pf9YTo&MJX3&@g$JNQurVlZu;#jI^Uu^e+vrGty=aRelS;Y* zB^@*3Dlw5q4>!?dchZ9N%VcgO%GSD4XF0f~#Nl-Gf4qE#+H=qOmh1C#ni3@>q0v_w5NROj+)!)1PkaO_dIE*>o$J zR294(yxiHBV8Ke}jvL#D9fpOX@Ve*jebpAJBL8ds_U*xdWQ(t{< zcyOqsaUPbNV+1NvAP8hed@EDCdHnbhj3}u>lg0<@XA({E*UGc7xb0l;19tYnrrR{U z=y6EO95Z$B&n5r&Q0sv!GP zy)J$=9T)U0-xn&7ld8JwkZH5SNJDd4-gZ& zDzHCr&<0p>uk^t8N*k}Z4YDmUFhBg)vHcE#(yvb@zAHtv%=oM(WvF(^)o-JN*dKdt zXLFDWtoLjmZj0yE7$k|0<@@Q7m3xg0NeYT7<7#rBF-({`6Q{GuIdp47UWie^d!to8dCvw>K$oMn6b`_@pR8e&Fq;v}sJ=#DPbA^G7G* za@x7FElCLB#VeuRg!WI_2UXKCJ(*UO2#r%Bn?AcYMn58x_#XG=r+2ltt6hlu#TS9D zt?0H*ePLbbma?w>8caRo33gmEje;e7vGNvG$urFxH`?9QrYB9!Ito`^B8mftvZa0RHn8O3>yhgYsOeF&l+;=ud7?Le` z0)$_Z1ZC+o>P(|>YP2|y3kwTNS3_;&Fpw8dMQGI**s`2nzcJJ}+l^PGnHf>SnQhpr zR%|MHF;owYsAVy1O?0W=9_@V}lce#Y->_L8>u}+Ml|B0I7XPs3`LS|XJR|P3pbn57 z2$4uJl?zH$14R@p#dhb@n(f!%Mb6t}n>NzNss|^5%*H6LpWVU5h6I*#!*=3L zsY!d4Aqqy?VU{1@`S$QOml)6M;Y`C6S-z1Aec$8xn6q?rf{exe{4cchjc#|WkOEDX zFCk0SXpoi%gR?FN01OlGg#@BQk+3zaJ=s2CsLd`qogR?z4rggJ zX5^N`Ms=)V*|&?C9gN$p5#CqInhe&L*rgUs&vut^r9g{-kU##8 z&xoHx+RkZ>U@Z&pi$O+rr|U9+U2)U(rpR#c<(uh#t5k)pP?cnXQ-00N;~WRfT{)5* z8?%NAk|sJZ9?ys3Zw4=c8#LM1`R>b2UKL zEk4=m;kIIb(omvpfqq2EVhXu~V5lQYjtfIt{^>EkokCkB2%}NsjwZO}Ok<+|_1MpE zJhQEnD7zUeGbhCYzQrfRy z+d0oUA~~opouXwlOu2k8vev$x>C{AkeJb*bA|RZNci-OYGOyMf97zQ*(drDpu%*}7 zQ040%_)6Cim$IXv%4QynWWfcUec6spMY4z7)f&2Ii;5eI zrf{rOJ{wJ5vk5hw?KegM{XtSrl>KrCxcXR1ZeD3KK>_jU>gm>_t^DU6F|-%&b4p+E z%}1LW;r{&=I-PS2Zl-X`{d?koUD%AP>IhaqRY9rB#JO_xHahbC)JbCaVfM-__4le2 zc4@k3G$IQ%91y&v#Bj!Z+H;gvfuISq;@X}`z1|LU*_Vmblg~2NEKfr%M;@~$9p9Z5 zK|5F&I!eq#oZdltKbw2ipzOD}SybZ#D`^nr$xH#Pp6@dmqQG1A3XsiigZG}Bp=A%H zp@Ct+#X+1mYt zegQq|$3q3haAa$gzaG5NXw;=N;2wUCmOIhLWj+BSgQ0Y+LZ0mCnJUx&UqY*&0zv`%_^U72>6F(ai=$gvkOv zudffzq4eo6xDAFt6%F|)LATzllf%YbTq+{x^ZuaEo0BdDeaaMo&5h2iNoF{8`&Tk? z1D~>B18x-CxnDvOMcCe4C`Hs@L3oHSrtYuLK&)_<3iBZBe8!MWtM$kWC+I$&{#s9| zU`yG@MO9`J)gZAbMx1ofbu=vLpN5n);R!c14d^ynz&f{EUm+nj<0{?WICLILzIC_a z(5&2(1gd*2g{q~2u}fc~!F|)sxRx=!om?AbEdTB81l!df2A0OHdd#cEnM^#&2g5lF zCK~jyI69t$r1q$p`u(?trdX5;OdWeZl(h8cgQnE85r&>dBO*lJJ19AND)9Z3)-Q1v!@VQ0F7^hBAaxve>y#7P zk6u`YYL8pfc@J$U3Im5UPKVsqi?`cJ0kTeEV#zybw&Fy$rM!ojObDxZCjlD5_umpe zT}5~8Op$L#LMtK~OPkF8^llGiOsxKt1wLCN)G7g>iDaCS^y_H;^G1w5Z+zC!3kG=W zB160=mk7Xy`!Aw77$-7V;)7S5hOH1=4j4N_!zA6=huvw3`Asntl}g~jC_F3uV*R`> z2w}tcM;kTR!{eH>KGMwo6$G3HaaGPEn zM|oPVb2N-L1a!}@3x<;TWJq&yr-ZGUT?FnyL?oSLD#_zQPJ~Garvq=7y4!54T;8i6 zrY^Npsc6lai3@*mszTy+46QrUt66s+(&_iFlgS+?lTwXsHigYEPn%1fzR@1o z58Uf?k4|okS7z-}Hkia|;YfxJNS8$iV6nbq>$V#4ZmK7Qi=@Yl7p}DtrS+(7_O5+) z^mXj2A)jmvPP~w~7do+mpGrhQ9RwT#_I~evCx}M!XO2kxm7DLC=hR{{#}3GPf^wUL zGn?JuO^fnL;13hTQ{T#-I==OHzzy-vFQx~`*j!V9f9kM zkOA!YMH0UvY!7Gh(-W<$2w(nqv*w5i%`f%|9rX}Mv`OW)$A7nUM%(RRc+HKrbh_<4dWw^TBhXC70eN##7 zdPe`F(17h{cIVF93s3T{QIb-E$ky0X#Y!S}ej#LT&EDPJ`PU{BYqQx4>307bR%OjARW(NFYm>?= zAU4l(JB?Dwk&w9cT*^^)t#Xs6-+J9ijjMD; zv`<$X^eK75laZ0JJ!FpPaz&U?@B+`Q$-06=V!cOU+x-OZg$zP@J3#}(EjL1<-oEoZ zc>yJ*?+u}~x)SNKR+%8R90c5%gc-~wHx^mc1%IrIJ^pAeHzo1A-=u9nL>yA?cTC56 zY3B;udIpsUlW2+mSffRriF84~hy6%Lm-870q>BA%SCs^`rF8NMso!6rz3Jgdz24j_ zZQFh{e;C2Af6s2K#KgC|%vT-p()A8Q7hp5HY8OmI9Jwktn825kz~f01qoP;+yh3UD zMnLqOr=`wpM6mDyLOPA~U@Qc6M~aHL;da~Fp&Wwe*IWssb`XS_KKH*;t~EgVi5_tg zriflb@rj00&ft&P>1Gmc6MZBON{>!#RXDITtHgKD9rA$iK}?sr%_&)lj~yzCO!v4H zP8I_2y9C3#6=6DIv_tySr8+5I&V?1g@s-Den>h$nK6;COn{}h;;wqr|XyoeFdt(tscsmFikpbh;g2ErBJI7o6(TeS&TF*Ww+{AIOQ@IvMtymG3$JZ;O zSn%+kN*=2bm~C;53)&pP%}8MH+Gl`wI;k0MrglHxjg+5n2+^VUoaO4X;r13$DR}{m zugCVVqoBrG<$!(_?Vk#T-)vH9GX=S>QKHDuKGUXM9H2FCqr`90h37Ss@??n-=q1qv;dJ^A-4DA-*2OG%pwf9783IhzmDoheYZ@6)wGs2< zMfY3FkG}=QVdCPgff66-$2N7#ANN$6vQSFId?6HorA{1y9queQ=f`mgBo2G14v2Ip zfzcgxTrIbq00~)_IK!%oJ@6|Ho+h2=t8a;P0vpVro(FMK^ilA<>0+_WqzqpXH(;u( zT-o#JD-0=AVVp|rwp6TPMv$R*bM%_ zE+x=308staYyri972xi(^;dPIbxebYsk(}%veuSsSp}4^kOkl7r3JB6(s90tv)9Wa z_6o8S5q*sSI*L^ZON-D75zXy!9WrDASspuka3T<@3K$sLomgyo?{0Dym?mDJcG(=T z7-H^hK$)9Fwh@qt9u^)RXVPyWd$=-Xf&t2l3HzV*%$1`y`8|Kn+1BiAe^NB9AdpV) zlmNW}k$2wiq~Bo7t)@oKsQ4$s2UM_GUB6(p~n7s885#WscwivI;bN??$aa!nJnGfnC~jXNEn zoB$FYddjAitIV0Hx)!~tqPEl~H#xulKT#A{{fO=@e zNx_)-q@!pAt00r-#S9dnYuQW4+_e)h4iqfw6crOfW>tE!cj^^2EkzSYY&i;2Q_(W^ z`kZJ6xyt&LR8)(Mo;jWuB+5X#IgJxN3Z*GCcg;Ioa-Erv>=UNfpxqSS67$rwb38lZ zm%}+a=FIyx!;;co5ecPrN&)}X0+BL)ap zi2-}!^Mp5*cIQLLz-Z63)sX&-7O`^0X!VK(&COcgbYzmX{=IIqG|+s8G`7+XZpGKl zk;?#E6<=eAXAi_$^T`(NCE62K-GcO)Kg4IgVt17&cr>?EHaNo8`QO!Y5>fmLbXQY_ zEg-PC$&jqUS08lHZ0LdO6+{k1re_aA){=bY8}`5PoEZO(6ElSgW17a-{Xo@{#Jfik zo^oGThb?!9x7ChdaVVUlpq+?`lPMeW;>B6*CWVu1IOC3$5EO@EGR+%eN;p8@>6g|o zZV@u2&Y{#(?tM-;TY`Y+Mn$((R!h9l6aBb=tDml592Ch_7(wweW%Ar_v)yDrgI__} zF6!~or&&&K+OeHZZ?^xW9;cP*tq1Ris9QYwp$@XF$j-DAC!>Ku#=s!MM$8%qVd$Kg zhQh?n?ZZk3O^t>T$-i-8OcH{mAqneh!yn!YMJ-{Wth3dE-G7;1tl?DTbtzTeXb$V+ z#Xsn*l#n+&uaWY6mqzjVlo%5~`X~rLhGJrPhfhgXcl4C}Br9g1X40#%CbSDq>^a4y zr1J9fLD{+4K)aMCBzgx22eUg(Yj&;iLUV=2*|^D80n-}eXB2Ch)@gS3y?PzJp5R35 z7Z=26`mgGZCN;iZ1Ucyy`7OK0uYD#Wl^T6BRT%Q+umcVHJe7n5yS44;V;=pyH%Are)Um_ld=?6=>SEtx{Gvsetbo^a_R@yM9gU~5k{@T%fe#DtYf7JDDeyFSsz$+W`V#QIlwP@35eUyH*BBL+hZ@;@7H@9 zlOt}rCRO7iKR+CATn?uW?+3%QK1pT;ypWx006Sy$Zx=qlvn66`Xh>qWFrw_OKhYPD zE?b&)_#-xa5M**9&bV$ZTn_VCODLznM_8;E^VrUbg@@sx6-6=-GT0lvK~_IHa*LY@ z)v%f<>d~0VNr=^u{W12_frs<`VlmihQw+wflqM3rs6r$5!tXtH1@|Po!}Mpb<+M|| zHVnaVcS)|Dga$rw(?r*rQd&7hP6^V_xM_=hv-YQbfwk8Qj`k@kO&#{TYglZ?ZKn(T zsiZiHy8~Q6EY~MLwI9Re^yXbvGGMEQ~qVSgD&lAI_FRbxp1NB5Hbiaw@8@jSjEHJ@CB)>x1rm7whUnq=U_H zv5H>GCw;C12)^SN8>z+^uL$62xwfh`iBmzufL?U&^-|aJ)`;=_okuf6OSr$$NzLvC z=F|J+y0$f)NYh+QKG&}K=YnzJTT3dv1dj%P!|%NUOcd9TBHC9I3Q*0sr-K@Hu=w`q zQHy8cV!>cW!yFHH073>SDqtsp*E0X3#b=4-0QF#PA;dtdvBs|fP`M~V=tCpvyKufF zqd&NM^nTS*?sb5{Zgx)cI(m6!_sRT?{=5=S^6B0b5-jucj$+HC;T55Zev!A-!oB_Bjp7L5kxC?4>*ek_jLdKho8J~QI zJ7E<8=AEB}oy*!nvOiI6eSt9=i&Z~X&r!!z8z-rMRO^AstA~T^B^bi9K;`v=)Ao}; zSruP1?s|B14DffNExw9^-VBv6x+|k(%LgCHhavW`q>+ffj#jw|;K<_F|1)MbW{^2hM z6R!alUYGOl#N`{-;TM1cN)e7XeS>tMDYJ3A16k;||*)%eo0>mkI z(oIEt?wLmWKD&7Xw(@M4D~ZfMn+PG>iq7c1_mS}U+ZMV{8eFjDQ!413WO zXZUNw1zh{b2*cGCtd+Y{&m(T0PQ1P-lYagr!z5C4S zWx|EpUI?Y>=}ar!lh=B&LMu`5duWm>qP`&0fY3*qHl?G*gXBiiURn%r~%&8)sp89s^LX9Ik424fM_=(;y&>%kaI~&Ef|C0 z^^+3k@4PqsQ!|D$5+BoqQ21M`L*IJ`_HqaELdzt}i9hxkeQU-WP|2gB8k)iTiXgxc zF5~gVMf&N&$8oX)LQVI}dsTZF#h?qYtq3+*@0OZ{IeRhMF7;s;$JYH;*!B!-B3A;D z$@QQr5+7%D1f9m*_ZgCxsTNynb>T4m9JoiFJhfNY^xTInf3T`BE~%4%%ND^&S3=cU zkDk=Qa?YWstemgeRr=oEL689iqR}LlWKBm+O#Up+H5|EIueVSlm(3WGV(91(ghquz zM68--!@iq#_B=Ah0d)PDZ<0108B%p_BgQlOjZif%G5zvvooVso85#EBoJP= zH4YI`o2(T8-T&-BkLZYOUr0Y#dq>sh3Q3Q6v_B~F0$uOO;R%xQV|qgQy9s|)Ye>u* z8^)oCKZ0*DYeLn!C!AtV3P&i|RXRH!{Dy)x*fciiVE*k!US%qCOCLFvz$e_aF8bGz zz6cmPohWB2{zZ7um4wS3)$biJv5`X+3Afghi+)Uc7mlv0do=#56*a>JVRx|Z=Y0QP zP*^@A&|QcMO$R5MmW_`y!}dpIJOTBTvc2qo`o0sQtH~nkAvMsI4kNgdV#3Gh6_N9Zn#&AKq@l#iFXyLv5d2`h| zh8g9qXWv;~cNS0T%SS?8oMoSQ=Qb}4x%BQKdzS)dA-Zq~=}u-%#u_fgb(!dJ10sBZ z-yxPL_yYkifTq;wVI8%Y3@YI|WJzWLOJ(TC+NaclbP?EA!o2(sMF z5>_|{mB+HO3V^BfG4~czr3E%#?+0?Yy4Bhd4+if{;<)f-(RRV2La{ZbJcBB7sKb*7 zI+GQgV3swxGK&KNC5Yct36+?Ez+mp@Wn4nyc+cVNuC?l>E`jKNN{m|86)s5$9 zgCw}D3N({LxU7*U((25$Eq2Ud5LI>%RU#2Rv|;#W&kHLzhL@F9+2&pH1R=LgyuHi7 zz@X?|yLVAUIur*)ol+cJexBg0d}GRpG2<1!={eiixT2Y)uXZ?3ta#d*F_5*WxvUD%=RXR*zB7d;6qa+VT_&jp^f;j5FfCgx{A|~&*JNW*2 zM%b{oCf|2YP%9^i-jJVvdpqy^AT1f$exy#ok!D0>WO+-$G-v?eeBR=sw0Xoo-;4lM zuTZbM<@zvKF|(35;)rX6kD6Z*bw4h~AgxkC*4wP^=L;Q$0IIsOgqKgNj4Ez(n@?9$ z!DsiHv}rt@a|>;D_;=Hxp(YTy8JrVd+~di z_)dL79kWY_g@qMt#`t^03JnY+Lqu?TV`X}#I(c3-DbrO-hC_$%ys>CZx2S?|-qs6Y zuC*+^$h)f2BFXU2)0gcZ#NpY*!~whs(7}UnaAVbXRCM{!jZKP-8ldqgtgq-s$sNZu1Q~U8kPeBP&el zb4ys=8IHPX!_3`>uwXaPtvWbV0l>#6U8TdT^L`=)%hg&3-UY0)A(oxuAl_sB-G2<3 zA`-JHrzbutFIcti$_M0K)^T-RViJdH3E|Mw8W=%5VBzq?894#7Tr={xTpz?nHr~$B zO7wnuQQ|E`5Z=x|_x-!%0;HmfCvwxP?!oA$RSI&+AtD2xuvjt2O2jlQA4%;lGj ziJ`U&VV7i*u08L$99e+peq$R3ANn-j2%??a>U%8{-ny$!vv%C*uoJgtCNEUXJQ9ID zSyD+sWybC^*^$XdWE4zxO(jfKwmDeB#KH}jDA#x8M$T-w#aPZ2xqO72ielh%;SaEm zeEvdH>Iuu5z+luJdSc0_1Uw#DpvgCUe(K&_Vg;{}iXd*-m`ow9&0w=FN<%h|T`D>^ zBMoFTP46j@?+=nz=kco(ihV`5$u;vRJ-H@M*Q<5@ZZu#LzK0%jw(QFq&9P1*#97Fs zZmParHgftcx7q3Ps0s(u= zyhxas{?^tqJ32XOG}~|gMxsn#cV+K2ZOX(Kz`br!(3>WAh%c*k3`tv^D71gL5E-`h zsO^jH38Ll&wDnclx!pq>%&c#QSjSwcxp*mw-r>(SOImY#A(dZ<9qmc}G0bf(QH1Fo zq;Om>(6%Kezh*bAv``Hcb&~QU7+VXTHttc`o{{L}Hjif?L$L5;!nH)w9V|9GU}Djf zkpbW}ag#;cRIsw8#I`i&+BMIH@8XP=GJGaWa=N5HktdJ!RLaJ=QuxqPJfC%|FKzZL`FgB1##@uYmLJi zGW>_eRy(|!*dk<#W@|t+TMSzDsY^|5p|I*yP6?q&(s4Mm7M4B+4yClwaqiM_0JztP z--=XIlcI`Bz{XTCH7Ax{lVGcFND%qDNm|C5Jqs>l+w*oF78dTEJrq`Qg?f!)%iSV5 zx(01{<)Z!XAD%k2&c#K7D@@f$Jt2mIA85Z+L+`8&mg}`Rs`UJNQIN5gJXpt4tJL2l zVO~;Jr=MD<9GJ42Fgkyw=4Re8Ec|ubCZ7L=psFqrL zWH}%Q!hbJ&UNTMcsf}v44-t*P z(4dp=jl#sl96VdDTW)qBiKmcjOu{}w1V^qrw%3O{^Ej4R&{Je`c*F9f`2gZz9v;7E zG*oaGkhplPO7NTnkRMQVVj8g(x%u>zxE(5#14PG;enOCWj?E1KCLfrJ1QpGhVYv$6 zOeD9uk=d@b9@Jzbm;&!D7-4Q!@;c6|`=YZT^K!As+WH@KC@RuChlfdw6G@XkG10^zy&x7fdwLF=5vCX*>Wm=Trx_r+x4rHT@ zYFFSV!w_$GNaD*I0*qg%>ywS8Q`-`f3B$?k&R@?;Zi#GXSa(QWm$j(Lhzz>x(m<)-z>;Z-g^8>F;^s2KOhjnl41R4>>55yqmum9n!I93?IA|qJ_^oncJTC0QZ-A zQhs2n2EQ<~kKs1t#>=uUR6FQEZyM|7T0yX>NKMYIh#Od9hrr-)pN2K%@hA-g=3pd8 zaH=@6;qU9t!D-@$jS~YHAH>h9`w)0M|NJ7A*cGDUy@78yH)@GZ_xCt}BPJ2X+P5b9 zc^u8P?b_R(Xnr3xCpu|x|E95kqeLH5Si>|`&-Rxg`IyDWh3;lL|PL^pxK#z)2 znoG-?G>rTquOyXe#aVYKk#Eo*(Vu{$k?w|IQ}^vLW)TVb%%95=uPM+SuB1v@{W3f} zyj+!7Fkj@mor7%l5B{B$IKluZBuvfbC+TE#)w1~;5TCr(``OXT#PJ$%#DZiN0sWP- z{<9OQ_#vE^-I@zHN1@wq0AkNcfS6LKqk3gIfwhPTPIG>wZWgrl<*S1q;>GLvgYS71 z9{S-Kf6(=AGP%Bl^aYvYnlj#Zev#Z*k}hO@>_t#^x>%|h^B#{sE3jZKL#K>sno3Xt z@)1rn=g8YkJYlxEzCk9cG2#s{X|&Ze~E6iwEq z>bAwtHCRDGD;k+YnHc9nNU|)w4!;)vAz?gyyNZKqm=Pr(|2g69@ZgrH*T;`SBv4&1Od!$<9!QRqxm46$boN!aeM0SFj+@m83I%J@&k# sbJXrEBjUC0onNyKHM{wg`~CIHi4Pj$qP27C7tljOR92)?NdM3O0jal_k^lez literal 0 HcmV?d00001 diff --git a/modules/web-console/frontend/public/images/page-landing-carousel-2.png b/modules/web-console/frontend/public/images/page-landing-carousel-2.png new file mode 100644 index 0000000000000000000000000000000000000000..bb48ad6f3d3f45578e4d6bc3ec1b79e814c098ca GIT binary patch literal 106670 zcmaI7WmH^2(=Ch>Ah-p0cXww7clY27Hn@9mcXxLJ!97T@0KpS9!QEZINuK9@zx(5^ zJBziLQ`6m5U0v0(cb_O#Wf@dtLS!f?C{#IFX>}+l7!@ce=!A zO#^J@=4t9;2_<0xHnSv`b1=2GRJSy>@OB=v6o!I=4F_rJy6Gw@3Yvo*SWW+yVfAuw zg4BkB5*G7vGBvlebR##jv<5kfP@Z-5QIdl!L@0H5l-QM=q%3VfvOX@B8a~RJ=00}j z0v42FqU6F}f{+FrEZt1Wy&UWvT?M^FDF3x92>Je3%tlH6uZWwS2<3l_(p6F=mjb(3 zlJl_gvY2ylvy<})uyXLSbMtU8lXJ3j@UgM;u(9*9uyY8q^9yovkpK5X3DM?aVI`<8 z4g9Yz$dd@Ajhma3ARC*frzfi?7c1DsnvFw1K!A;%lZ}&;1yX{=)!Wg{)QiQ@mFj<5 zkhXL+cL6!MfxwRBe_J#)1G~G4P(m#IPZu1Vl$8FrVMo{hItt=4HZM~rHV#&HHV22l z{rXqh)lJ><|2O0RTH009+sTqm-O?58?qUuZ4=bwwsSNSm|69@Df)Hy2Ra`)jK{2(L z2AjJ(SUS4NNsCZIeqpr$SqO5m3-Fk6@LIC)@bO!+aC4cOv6u<)39#^6a#>k%^IKW) zTA2P%JO8)(QUYA;TtE&9AdrWjg9E@Pz|SKg$qAI?1@iIp1EqlfQ&-N>)y>q=-12{P zgCM&9TUX%!SyxcX#nRLb?4k(<+y74#sM>(tz^*o6CvqvNf1^%LuViWta{Mb~_&ZSl z>9@3{3&_LL0_XyEAphrSLD2uj3JYFdGk!h+3l;$mb9RUa%=uW%c+3P?I5{}EO?h}N z*{wLZDF0jE;{WjAe?RLmy45<4PqkO-{AQ_g6V%mkO=+z^?y?X^6KcTgg@HRLP6*D%1KLT zdMzL4A^PC}o<;(fW~CU5?#j8<#H9fMX$hRhk7th~#-B5EB#s*@3(eScjgOIerJ3m& z$tmf{dm@y5T;%Fs_p-Wrjva1KZ<7^dy?g!QCr+mReokL+v#j$uew}z^44Xj@vcDf> zS7c#dnB#5y*{ZGkGQd!nFHDO5-RjFT?f#h){XvOQg>uR9h>bCD79fv((nuhZzMNV^ z`&8x7QC#&!_NlhUWucWdwzV;P9>6@U*3L^X;X+UsquX-y$vsZS)&9?6!I_BC(RxWP zEY*^MRM#Ex`}vU%N{_aTBNibmCup7Pge~&5wY6?hu#J@}+3QG4LPag0k42XbHmDJ% zAHq!u34NyboQj8JADyQCdK?D*i7@N0bEMOjj_U5t^q2bd8LInk(gEt3K6J`?z%x{p z9GObt1B1N2^^Ik2pQD>`5H+~uY5tYOwxDR-wA9dN;$|^ydC|r=NxCtqJ2qE=YHxHu zCm~6&CdP%JDzIcHTC?HM5;%eDSil2X0^aH6xZ<4f)W3XS(Q!B$kj;r3?y2BiGHAZb z*IhrOY&mn zwajn>yD+X(+5WkneO{sDgai2|)-iHKD?__PutgXIpP=2p@4j`A$1M_*tQ{5Z!xj6j z6JuO{0fSJ|KVxEFl*slMp-Y~#5qE>@eCC8Z`}pF|GZEB3hQph>k|Ke2O$m>p=sYc= z0I58K{ANmdrl43sMNZT+r80`VjBkeZ;rbq;GN@hlGmqs?PoVWXTZbNiOCmZz>Y6To zo|Z6Whe1|&A@!4U@K6)QklDPN27`_iWI#{9Fr?P>-u+5k0r)TKxOUueXLzhZ6wgGb zEypXzm8sfZ7cW(QK;FT?n;5TB%E+5>pc42KHqJJ0Gbp=LJ$Z4&BTDte?Gp<;)JM|39cYk0BG=4I)Yn81Omu`)*1VSZFJ&h18;F{AZlNycb?x*N2U) zkTTzJtBfLuKB&(s*a4&h)#z6xov@$G_ETiVrC7ib%=hG{k@U^%lt9Z4`mWKibGr_Ass&3B4s_9)1 z(vBCWmCA|7>X;_ioH%3q`3(4xN-d^kYsXSngA<a}(EQI_W;{R}oEX^NmxX}i)5MX6iY ziP^$}6*XI{AjjW1&%Rcc2a{{&ei~rFip3e7IfE^g6dIpYn4SdlaWO2jX%YDSLnHUX zP-8MY?%z}m#S?7A5@`Ic;IK10QA0}!#FYBOtiD{OGY>jKw&OM%weff#Udv3L%nVd7=+VR?Z=_>dXed&X!QrWAY!;+eMC74n^2Lm@xg2ZU4 z85x5o9)l+?5#KjXoQX6F-P9{7wKo2+C(|w-#O_+crEtb_Av|q&R_3^n)INeMcWl4A zJAYp0F#%8R5$cHbQJQkdD^`vxrTL?&sTI%ezR+8QQ)pu4~Yz?z1r55Z!QJE<;)rLhRJSQ;8iuqAZpvWZNnD-EmE zc3#4ioWvRyCNV1?vbJq*npL@^q@ZW$BPe3^HHSSe*BZdu#h(a)~9 zhCP|e(8%sH#7HfnboHLjiV#p4VgsHk>b$I+Wn^Sj)z%MhzUJ=uzT?6(j$lYMqm^=U z)R{yUi^!&Nr(35jiIL(1mv|G%S-WMasp-r)*mdT0_NKbGu=K5L zhD~hZsYH$LZR}#TfiG5_^jA@fdn-k;)|?j(Dd#UAZSmcIA(-j0mf#iYN~1K?*laC7 z%O1~bemsDmnfG`jd4Hs#dtot@WzZlgFE3x(pyA`=^U3Wsp1TZQBzAvgC$_#w1Ha6r zc2;hk1RzWtqj#!GaFm$2z3=#BWMtfeY9RjTE&p2b5lNSKr7sKLka1&N z4dyNJ;#JkTl}M{-70SC!tsM?G01 zF)0c2^G5Co$6cp38m7RbEci>tfb#2p^FQ3e8q=^pnSga0M$i>7=JOdR6=@=N0 zhR@u z!@qa6|%b*BhMUnk~DDBDGLY41M>ZJj2>B zw*jS&ypNpRV&AI_cVY)$v$rp|o=&xEFUQ;wg(5=GeDub4r( z_A2^J{=#srg$qARFYEFZfDo)Jp}yQJ1>Z5)m$`NvVthP1{;Ped#-}^@Xh9humSp0Bgxdg{gN~ zs?ADPhJ?52ds|T?UpPu!;tT_%o{QT1)@(XhH`H)2-YS3*F$g}4jnd`yeaU? z4F<^H>y5e)kl1csw;!CkK5hD(dk(&6PYJb5HZ^gUmX^bHM#p_~GCkBgk3LfH{-Y2( zn?1>GL>X0XQ!CHY$Qt&AJq%m<@Kn6aH-S1zq4{c&#ox%aY`Cn~IxdjRv4-XRIEp&E z_#l36rM{jj2Juj0S`c&colxB;Zifc6NCx*B9=Vk3B7LVHXM$yU& z+8dOx{(Xt(3=P`-vq6 z0K@L>jXz!9dQx%?(-mHa=Hyq>I>(vm9L?UnrtvV&emgi9rR7-zA263#I-5qEIK`b7m0WsfD#hW;b$n?;k~cZ}-C6lx_@`6ry8l>VzJDnkS~@eWD3u{(5TE1@qLDv2wB2?S1K) z)9S#nJxcMj)qyEJ=Be0)IZx0XZ8?4xRqRhFAxS}cii!X{6M%%#o6numaNRb&B0PNX zAX&0rt44cs)iYUyPsqi!QAIf8d&}q7d1>k4Cdkghs?KRvw6?@~nTAP2Tv}E00%Psn z@FtYTtKkW@a$ZwVe4RhpV-DQ>s);3G28~)dU5$q1DP@6^?aapA7$IdJNbfNnZ0R!` zoop*VfbH9Rg;X)0|COxqIR3?iV%-s`bO{*|%Wu4~My9V$zXjQ3I_s0@%Ttm6({1kz zxcXGShgFdVeYw}D%VTt6VsNp}pk|@*@}ipAvq>>v#L;2BX(pLRfqV6<6hh22hTG9P zDr^Ej@nAUWp!24c`#==kX609z;sM7DP|K70Eq+ugr9z3pBF-JQysXfRWEMy_b$Q|k zC65+zh5K85`h*WsQfb)Gw?ZmJL^!4Ce34I!b6_m;@r`S>P?avUlMqp#6-H2tw~^1U zvdfZMo%9r0!M$A9A3fHdt8Q0HE-xgWYf%9aH-kunDv3uRg1Qp!j7VWw?wN>JZLmzD zlf10_`(r6z;c;iCgBEvjP}+22NqC`2Ux0*n)f_0Huc)NQ!)R*!hnXZ-WYsKz|4-O9 zUUaRq=oM4H#8k7j#*8$DBMo1Q?9sk4W2=mMHsuKH=w0W~cUm}(+@MO|JhiV{WN?1@ z*H&6+T3&Uks&EmMh5M#w?pk_g^Je`-8I*k~0dEsv9ebMXoAv6xBt&LCXhSC}@lY-5 zZ$%ETFnx6zhM`j~8zZ}2PeMKR&I3Fk+PAf2qrCjHcj(S7J1BxwKzLHjnxE~`s;eN zTTk0ex5IhNu}#6%ZugvH^#lKX3f(ptxU8*tuH!bJRFDNSh8nP4pRDjD z;=Au*SHuy(cJ&x`JCQ^ffO`SmA6f@@<8s*jAKesv9R-z%1B$=z*6Ft;QO$dveg}Y@ljgCh3vmi6#D(zta0jK^Nz+teYZKsz4s*)eP+xXc)Gqw;M}7U`cul}@jb>Ci`%`mQSzro<_8Y}wN6eIe@C6#T!dR4n#5Ea;(0~i z1q8Hv*Ua--zl!p2jBi9IXe>Fdm456vS&ZV}JN`0Lr@&`nWqFX_HTNx_mh2h*c@LUF zTbV#aKH%+c>`lI?(ftaqBYy}62_K8j7#oHR%aE<_J{jkOb%B4ppFoZy+0k>&&;5WNgcBaY7}TpTiFon5)v^HhZ=v3>qT(VTS-=0B1tjl`Asy z(mhXASS%-l)9e4a_cX-$ zkn4D4nM&u$Ix`Qf#HV)MzRnRF^Hm& ztzBO;D=)j$(1CZ)!ks7~u;rEHArw*Gp@cVb;p8G(Al`*D5MJ5%gTh@f+; zvLt>Wb8Z9$btLSfOgA#VoBFiR&ngE@javLjUV z8xl1e?!>JXDGhvjazH~H@h73%0o138kHFzi5-^A1p$s1u4`q*Aqo4D>D*uWOOuFTl z6L&vcMY1m7B0c-=r@)&yr%Xgur@nkDwDYGN)lH5JOKGLmK_+LhuFRFh;b>MjJS6uB zU~H@8$MAhhX!`Q*%UNs{X zdr4b^HtbQ-eTBXa`%655RK)3G7$4DnrNv=M8I8Tje^%t^%d}|P6S2_g`;OR4_3?5y zRP_u}=YyS$G0SFjM%bN)Q=v12&Ltb7Fe6HElV+UIkzy8NT}5}=X-{lVVaL zbrE1KO~m6a(X!-uf9@wabXkiesOv1Y_@q+cWOC03Sc2P}48gYGuB3^Df9VMRayHkvk=!*Y6KW3L`R3X*6BGy7RU_n@JPoIZ#ER+PA1;n zKZ#<6E6d+6m*Y!05%M6fuvBo{|0#h?-11m{lJ;kBn-4hhr>oNN(rb2hHsra)B9U|E z_gys655|jc?cg`jW5~j;WGye)xj!2b3}rs;}}?DJ7J{uOLQ}7#z=6 z46N5jjX)-Jhu5dyZ%TX9z`+23t3m5A8z@co(f~gzJXFs3&a#X~9jA8Vvsz?SU}M0G zkN0G?Np^H}%&=6&2NVAX8b&5YH8tblEG}z#l=);D1&h0-N7N&ZlPqh?HWs4|qT!pD zC6ho3F*fv-HZZ!MjVel*!o$r{n9GXW1!s%d1M>HrI~I?~8@w-CyMU)hh2{dhDwfh= ztCLRmruS|eg}#zjz;wmCTLUgjg21PnrGC!kMT%4Fx7R1OWUS=#s}T4mX6okAI|v3S z_HwO3?0k>eq_DRF-W?aN6mIUjjCS#6dojBuw*S=)NI#e^3mi9pyci(^8X0Mw{4bYgDnyDm#YE)XrCv?bJ;y&(Y ze)y=-d6kY;fGbMmW4rd&=*DQyIpE=F?Xc%aBDiCfU2}W>O`- z!gX%^L&Wq)3O<|P;>W+b+VpjUHGpvgYElUU+lX}VB(9auH<$I39E1+mEugKW<*2UyERYT0t;1Thu>ZrfvpRT26PcNx+Q4!$+2%ZG}f%x(HpkIrrIrAi5g*tMa? zQm{{Rpwf^AJ*L*)rW`lF0ySBM?3%l+%*o~eWolGgO!_aq%Ob^1T6>D;v|S#XsTw7p zOR`x(%bK|vxiUnpJ=VM$rG{cse>yg_G=6m$?bDx~Bz4QpK;@d;!ZH1{lWS%l`Ru$7 zA0;bQX9SYxX3Mm-95-`pa3e_4jp50Rkas>H?cMF! z_uMGRHkqQrJ^s`U{rw>KP`#l+;%81QcVW$QMG?eHCygeXFij<@xD>DComh?CrcWl) z9c7ipoI_FmHB_-Fe~LUedN0=j(|RHj($nR#c;%`kC&gL2GS-@q1 z7iAC-7z23v5zpH*aW)SuW=M5Xzbr#A(o_$rGiV)Gy&S&?m-^O*1*$x%9ng#F>AHiX zMdIbOCU!7^V42+^&xOCejX_3Ohme!2QGonCv<~W~@BT@`xPmq(L_hq)!6w20#NK7U#TWMLnCovb%nYFjZ1=GV^ez zSrUqt$@A0iSF{CVbqxdAqF%q@=*0Lu$XE&EWT4dWG`|cG4ty12{ay%b5a}y^?{K}_ z`vvelXDHlH^Z^htd{}_-vrSzYleSLU4BCC6Mb-}UgHlB+_kmuhBobX+8Dt)b{`Odk zt3RV(BI&UbcG#p5Q_-)gNRcmV$_W1c01rPaUrd)jj~@Q`+*SURpbb@3hsd*L-0+=e zl1saIR@TRuY~=>HOx@FQnd|X>uDS9+`ot>c0+d6kbb&L8qx$DLZGKtuCKdp2E0G->|O$UB6E zmiSA0-ONF|9xfbI9>Sph+pvW^N{gVGQVEk3IIPSMjmy}O9Z6TRSdp!pU#;}MR)n7F zY6aBgkHa{@RN5EMsf9Ag-dNF;dlY1v4RhUH8}|;GE})rhgVv!E4t9Wnd_H#}EK0q; z&{961)au_tdGOhw5k7;%B2T|sUa zemr$GdM^`31E6^NNbr|JQi~$8rk^SH(X^vXP5~!{D5Od7Q0%^E>6YDY|BcMln!4BJ zL)0?r^prQUkk0O2J4g<_RhQkM98KT%`BELu`-_~8FhgJsRhu1r>a8a|1#XNgy!MF% zLR1+-lfvGSulYB{pva)BnWGjaeu;8yy*QP8`IBl!N@u z!$V(`#);P9YDJ49Ki6%Y4gHhP*+XWrm=wIaz7&HU>8byfDZR^|9VB@tCmx{kuR6Pu zzAyk=NfXr|4`SViaF7SYreyNpz7yRmS$ZA!1;6gvNE zU-&uG01{mg4c|IXu2nr(c;sqQmQbEw5!pAHf0t#6t5de&{p(7#%m?&{b#CU+<-=_c zK4=QS!1)+{kU0=qp%-=vW5b!(k>#J|^Q)IQ$T4L}o*QbJVTf-itG+q+YRWQjZZ;Dv zVSh(ElkviWTef_8Z>+RYn`f9^Mjb%XIe*wf30Xn(STXYo8tPBRzkfa%yTS931mP&86?RSc3c0uCtI?9c}ILoB!C) zrcN_&)lA~P1HU?&=Lqt)<%X%Ag%*D0K3{9--vdLxC5DY@_C#yfLG|XN#}binGGU2b z?M@C(oe5m6KR=R5&4gkSikdSq^xAL%eJ?-`_B?Y zAL+?i)mj78v0%S*T*LkjSe&n!m7#Qf8m-9t z5|f~)nDF(w|0QUZSZz2?`tbWUI{t4dIy*U1+H?$sgbHmW0F=HRXlD$I#wbLO7gS~w z`0GOoxY@Y=$t6(rx8tYoTNo!VvEQ8&l&2xm=t)UQ6f9~cfiEr174gv6usIPps;`lzhA8ETC%mP?7P=x>GRq==gWN&NSuq3&N z-m)Y2P{*vnOiZ1=%G!pb!CHZWRzXFcPfQuNvg2C2o|O~hY@Pp$J>hXUs~?rGI#i8- z-HV#t6Hk0|^ExKzY%g+WxpH$Sh=%C?GO&Li`_p0~PvmnFgu0^+L?;#g00~W*k1xTn zh^1*!qE~y<888ukX`EbKJ3d&N4-XHQo)D6d7%gHoe0GKx*^P9cudKp9SzJ9+*qf8o zXP@HX?z8FOIpm0-5QD2%HOk2gnx%gDm(raf2!LV1x+J&8?@a3YQt5Omndh+4gYBDF z^YqLrSNM@{j9Qzr@5;->pr*TFgnii9x8#2BYl*J|8(E>XdU7axgvj{!kUviO_R9^v zKT9V5I(FIYaiagrzb)`BfWfBI<0QJdnVTVVxwQ`Y8v#B({=L$gSMy0UE>mfymUM;X$OWPxEK9B37p>@h0_54 zeL2V(9;L4C4T^8gPL}@#f-?>;=$IHO@nc$q70-ALMxc*qsEFW3!fP2D#}k3O)gJjk zVb~H>=ck=VtOu|FJgN|8SqszRq`+&W;Xe^%@wX={3V*m=wq?8TH)A?svL&f<{cnF| za#|vrFksT_w;_vte|S$8_(-$MrXS^}Q|og3@vWHcqTictD|5!$^l;$hC+C}GZ0Cs{ zGmGtJS|4Eo8?M8&we*HPF4zSaL$B6?zdf zjro=&)wY;zb%Rt^Q;$IEg|)wieeXx>^aLAlQHrT*5}or%7I@G%@z^yn%JJO$bc&*1 zF%(Q00p{`C@NaXwr!@|IaX&aF;YXDGovd}R8N-&YypIeyqK}~Rt&+c$JN7j6KFWWK zEaq3G{yv{h=VDgz;~lu~r4>iNZPf*Z*I^mQjde_>+VgB(V#nvWape3fBH8&uu>uV* z^XogEd=3O`GGDZU*E5<`?nAfS*LmjWMlpn?5MMUqybEFDyZ8ND8f1f(GM+&+75JTK zhcVl>**=IhUDVV;J>0t_|8C9Bfp0sMx!|ElB?= zjq^orawrP>;^l5b!TPjJA!9tr%eK-t;W^&e=b(&SO=^+h@kgXE@;3pC(Zv2o$XW(K znAZ$(Va()qy-d1#fZ)>c5X@ZO+5>sMirwkAxL!WG*t06KXtYKdnv#TMeD2SzkK;?M zUo|oeKcULx2jhzckb%1eu+Nlof>mfC zjb2ZWUuM|KK#Nre8ymqF8Ic(oSHmddeElO|f)6~Sj?UkP;5msLJ!9`=L=(0ggYUYN zK0BvaefU=C#VJ^k$2qIqTUSLH%h$=TVsYWU!%%o+x{2tX0%C!qmK<`pu1N?Epo22; zLp1b;X5S+rZ(;G9+CA4Re$_4NLUDnjqdV?* zul5^s;CQ!*=!pMbUJTbrUPRQKsCa!JlE7X4fQ)42>t{0pD`5|!Z*f8>I*M;tPc}Qf z?p|k&4EG=<3dX&@(42y2jA65j*MjxIW)RrSwL|PGx$b^rkT1%^MesD>K;)UzleXON zvurwe4!^>tPH`YG~)?VTsr?xD{65Bh})`sEQ{&xAh? zU{^A0B1msTCv33PFYIWgSFKG1=Y+l!+Ohb3df>GCq_w2e0L#`*QEz>Lxrh9A3iEo0 zrN}nb(8SUu9WZwatt^PyzK!>BxW3x&F52#KYC^h~Dwt_VVX^P&LH%RqE`2`Ej0FVe z7(V<0%;c7=QF{=^TpLn)sPPQ$m3bIeEqk>2eKdewOlu;nJoN#6l>>=i?_E_}0XE{s z09$9}r6R?91U!PF5%%ltQ#hdZz#e$lZbUa4W>oodG?q3xl7OT}mPg%M$_=L}LA?B<{3a9r)h{RAj28U)>eJ3rh7-m=XKl7_%4SZ}pH zwt^jVVY9_M!uy=PVX-SI1_ z?xX1U;a3k6itXKL*Rb)8J=gp7%MuLWrjHTrEZHLJ{VDFx`c9-Bx18nAc-7|{N4}S&H+D`TApDNj5z{>(uV~u(-<9-UFD9!6mFU#BTiWlF9<*` z%v8LKYWoqj1T!s+@N1awbD!g_1^oEL1PWFu&U+1&@7D)&h+nLy(OzQlB%?)b8%yIv zicX4A=<-Lia%&9^yXkeUsp(H~N(M}2zQ$|GstDjX+bY>NQPjDTjgq6+vv@rYcT^sG zajd^=(>CK)91Ij?wLj-~olicPJ4vJ?m04EtUp~s5Uy=+C!yokj(TVGy2=t3tyAz34 zpcM%YX8|KcJN7&Kgwi#w2+h z66juC!SEv{)I2$|G_+5r#vJ`9p365t3SOQ(I&$Dc`igDhUT~p5e3L{jZ;k1`RTTg%xZ?G|pmNcT2zz*#~@_@qD$7TGG@IIN!hO7cdVp z2=jU@Xumi$g3T2Gnm9#2&P)kZSpO<%-lE%TC6%>Na=AJSHny|kNTbFib-V_1-+pHlPDR{F5%n=HH}<`IIyq^y zycNo9rbk+g7G`E$4hs=E`W(Yt=FXZgOB|Jfn^X2kytZTJjj5TReI3!ZEWjZU_F=ih z*sxWOz327w4-Lu^$qU~>Wr|qd7zG2vh_!@}3Ri`fE80UGE9V|IKiJlVO=d6Yt_|w= z*NPh_6F+9=k;rkMOBda?m?!Z9V<)jY+C)wNXUGn4(hvehFid)C`Sh2=tm#!DqiHZM)FOe2alW~KHkCzHbcjY>3ibO^8nWz zs1Y z&#lck4;~cE$*J=7{mW!x19$PHne5!~ykC$UZ@Lp(Eu7b=BOMGy!9ooNhUU>+z4(rw z2FDbQM;+Z}dys;UhbLlnw$zlgi51ec6y@WC*g3wc>oH77$bB#NZ+B@*S}dojY^c|cLbYh-XVS18bkZ+-SY+59hPjj z??O>Lg)^{ot=6Bn1PhCf%{PYBpP3T$CXDR&9N!FgPsWrOzf-e#fBC^Xw_$}Xv@DMH zq3qosZaO0giD1Lqr?bAT-5hW|Z9X2Gv0+1mO0ObLyf}e_%w9B%eRAP!llcf51Kb#A z+4qJK`Z!T&ht4~hj4|-mN|?^?bB*VsO|W3l0^tfk>1e19lSY&jO@_Yhx(l*_~`X%=-_U^Wrm|aEMZXi$+jI#(7#9*4x*LDrPdH}Gjjfb zj$7S-;KH|BBHCPG!WYd4cryMiMUfny|DITE3g4%&#Ouocj^_8U4q&IWcsy%$+t&aW zeAs?N`0>Xd3Qvi_$1#FjNho?h2zAS8)=nwOfU1-wd69KEJ6nop8?8zz=w_oTcO~X>&j@z{CE?ZFC}BXsi)_4kX~~!baZC^)zkHSJ}u*q z+?-9Upm{e&b)ECzzOwA=sa$N}v;Xtsf64*)6-T5q(qF@9J7X6|@A6 z?g+1HW;MqQezx^Oe0;<>Kio6jJeaxbrQxtEXs3LyuMgt2LJrK`dYhzkcQzF2F&{gW3?mAr zvDie=VHDaTCziLdKhmvJcdcR3t)ly}7$+O|qFn45L0cD9a_7iu9EOjPT;ox#VNj1m z;nP1sXVT>(et#srKeYTgC5+SxzX_P2++NzkHtpH<(CyWO^6OKLY{nBWu#d+aW}lqA z#8ER1gCI%=tthSKUKrM2cvnk$_w7G+VgQcZYVMuf7XX6k% zJ)oyLdzO>->NeMk-}qC|IVU(i(ig%R0aQi20YM$9EA!YWfSc>(JVf}1~&=NW|* zj|R`roA-YA>~VCz4QT2L)0Vo6LQ7S3)%41q!JpL>^t;`O^b2j$lmnXD3#DHybf99y zK0Sm`h(~Ay(}n8#Y_t64pg8`PkC4k)51G~4<(1e`#mF*`;PU+dwQs-Zs(~9&U=%eF zgMc_2ez&vLn{&(?mqCo(3Rq9+lva8sqJK~$Xpc5(2km#Fnv**i(XQ*XKUeto0e-Md zhI|3{>LWKJuo=;FnM0AQ^KJ$5Hk{XIgVD{x(_W1Lris*|bNO02EC~9=R4Ft~G0V^B zPlDjG-HErK6(|b&|LEl9d05{xqk#(U|>C#jQhd_+jec!BW1efa`rb`VsKlBDQ zEo{ir1$l)bmXHTI29G7Vj{z@*Qjwm0w;41ii5IrXSTzWx>ULoO?$JV+psw#cWF&}? zbD$`Fx5@1uEgU8JeKM0nD*aEs?Iw5W`CdBFMZ1SMcKv!x^m|ag^GclejH_AI16i3wxc#;zKTxe-A%VrFwd$=m3;5V zw9YCPK-7Lj@o`(Azc3{zRQsJYsmTS3)AzB>Nj}`n+jm(6L8ekt+U#Mps;cNm1>B(I z=Y4_Lxm-^jLubG_szIbGfFBb8SfJyenn>cL_{>>gq^GB+N*K*AYB~^Vn=~Sap^-mA z>RTJ@ri=}BB+;wJZOWlnq{~m>g*_8%Hb01W71ZQt|G24^8}R4G>-wO=X?ob&d|aPz zs&8Aj)Q0t@o!-;I5sGVDUrBU^%I@ZvDCj{9x`NnX=MBwdC`7dv*eqNhJ|LIRMv;r9 zx+XQg*(a*Y4+)!A;ixvtm`l}#FTzRFDYFh0y7`8z703A+%qrV=+t;0C=FTL*fF_s7ForzTgYOs_a)a6>b3qPf^LauYsDe2u5h6&~B#rwxK7S`<@zMG^MhnqO*tCW+aux{Do{tkU-^` z7rpKaiD*_Iafk0^@cg2Um=_MKNC%(TPYA9xp+S;1t=g8C?=|bO!mt_Dc{2LgY&oHw z>w9|tZ4E{qLUZ^A>8vB@x52!gnH5FPz6UE0J}otvu(3!|o7woJk&5`hx%cht?M+x- z*G1OvOk_areNOqobE>U33itkbgy5gd`G?JZYGI5&sOV+O=hnPBPhu03-(Mdi2=fpW z3j8U?P;OcemG!xvk+EFL)U2k(X6@L<5{7^poQqg`RrMXa*+EW}qgh>m7(Bllj+w!M zCKS~z@Tw<83NtC4-vCA&s`OY;x^!K(n zWQwJ=$(dm5>t3gd=laVgV3&DCdV6%yX8dfZeg14Kk17w~2yq6U!c ze>KtA_^YcYzZYQlIMxNlsUuQa{v(tAU_y@X?-+XesD{1O6I!V}c6WK}XFP78hP8^5 z9JrU`&6s8)TOmZw0lRX^*}p1{bN0mFaB+&H7qLnfD}#{dzB`O)Mo_xBraJyb_Cc%j z`a7Z7EAfBL`cOU=U13wF7`kZMEw9)($p4RXQ!*50C^?ag>(r3Mt&~~G;s)U!VH@QC z<1|2*n+)X1lDdVds_=h+Az8J|+5eH#ZdUQGdFU6={i6>uY1#5A;m78s4qDCsE6UFE z;|0~3i)GJJVx8)kH5+XKYJm6Vvxos^IZ84jd?trLKn*;chIWgm;dwfKm4Z#Fq|xEl<)kQ)y}{X#Bd0?|Z-f9>Vwkrihk`(k2~sH-L&Lzz5ZEX!oE5595PF~AhJ zH7zwRZ;MNw`twfNyd_EZ;zem=yjSk*N88fTQ%-NoNSUCvi_WDw;YC}~jK?l}fFkjK z{Y8kx76KQg-+m7#XMb0*-JpYZ<5_u=3XepegzjJ4!jj<8oJY7cjQ@Z`{-4NzZR%IO zGweUKxJ9MZNPl`mW2T~t>GPc~eW-JbQM=32=`nt&do6d^em`e`B5`2wX%b7 z;&LPUymQS6dKn>~dq_}XrTSBP*vj?yjdKj~U?}D2o2cG2^gnKbWvVV+e!VKJ85;Uo zqxkYj`3HsEtQpU~){d--Yn==9J_7#?LDdxcY8|E)sPObY3i)9No6zcQLp2?=n8kAt zB3ga3e7)>o(}$<@y1a4d_x~~V&Vh9{&--xOw2f^yJh9C-XspJzZ6}Rw+qUhT*hyob zps}s@^l3le_xJC8&fdGTJ2Ut0%v{$V$Xj|QUqV(~y;%-j=I>vTD(0EBWNyf_cd7kV zvICtTyeq#xP|i5#8!mT|yhO59ShWby%)Z=KA=+U@8Ib)T4v)JTE)y#YZzM5~XWFJ? zW}e;G0RGWq$vAY`EujSD9`tdyyr&3qYm$Rsa~ujjdag5%JSBqYqyT{Zk=}m_OH~+N ziP%85Y-v^-l^V}Gbcq%!`i)6U>A-OXI9)C9pWUcv02dZy^RF={x9;dX$wN}&3k#Xjf^$m-O#i7m z3#0={_(EY@1WEXRUQ9lTCD^J>rT3pD%#p$8u2hoCb$1Xo>W-rRH@l8vd3IQcRLw@X zK-2X9W$O*$DDmswf6M2HsQYJve{9@wB>-%J{!~+SJSi?ebp8`UPtxz)m5HlWnqIV! zO~GgR(!h!90q(C0mB;a)jEjK+ZGBWMZaHZu=j-`9NLmL!gFpfLUYD;F2V<7|pUVFF zk<@ILneP};+NjnN_+LFp2qnCkTVi)F1#x&#+RCZUo!EcU&R(0K@yWE7 zPX7Go%OvGVidX1jfnvPN6Qa?Q|0`P{1;z=G!p1M_YJ@HSEq8#M%?HX`ww~1U{j0VB zPXbJA$@hcC*~kBtHyGyr;B`YDC@a#XM*C|I8a&u1jm#*dvwHu}BU6!;U;U8c^rGbd z{YkRi6v;Ld4Q*N>?n{b=>mOynD&#XkLIBl3#SSO_6BI%YTw>#drA!aY%KNwU;ZM6Y zrx=e6o>MAn`RvV9*-v;!fBoFvPa@_R)ju9hvHO4P2npZ2UqVii2sr)cJNZn(Rbv8M z(E3u~Pj$_|H7g7Uy8t7IdD6_>0;}S&4Il8Q3SVOWG;nbArst1BL$<9@RYs= z)bX(1T5&I&I;vD#z{61SR;SVq{hlHmgggA84ne8?dPWG&&1Vk!^2t}9{*s8c+6w!# z{-*1A$5z~BF+v8ecJ8aKN8tId9yqXKLAQ3=)nCV*+&Xdf)|+kAKwG8;+76J^IYQR+ z?1*YOoZN7c1in(|0Q7X<&;2e*n#Y&jaeH5w-a}C<)R# z&|S$89C(db=m;3mPFI1+57QpU%Qf$D6Dz&}>j5^?(UFi1nUSm#I49<6qS;ptG7p zb}X6Vn~ni&DYB@Pn@GeDN30LY^#*|XD(a8oRB3~Xu{c4QyfYV@Q%+d!CK%g`T~Xk< z1ID>Yx7l>c7#M=x;14m}{gRrG!z|oRy7SB%x!J@7FB2VT&TBR}A2RHXsMW3sLsm|k zNbUxnbhNK9wBD}ZcpNWt7|{3hf5mEFRF&`sBA!Hnk6vpB6KlR*;EIaO{!A=e3wqKL z8lePtcAbmbt{PT$!~|(@F&^q?Ja^K^reTG~@2`p87f;%0Q)Zhw4m3+a8?TSBPuolk z4r8hu4G6JDpgr7VP~c3NjCLHPSmB`p$e*8W{QYD6WXyD4upRen3T6zWLV| z%vY@@^Z+NoScvJ+;VP^MJb^#k@W8Q1z%HG}<&xW2R#O66vI!@7msBA_cB3NEGB#@o zsoW%M?PD+#mk>b$aok5-oibrn>Jyc&COFKuR@A|))lYR3hau&p)Em5DQ%bFK`5_cH z4Fw|2Hb@8Hi(Ce-TN+a@hF7hjQ;Zx2D)rF`MM|gE=k(c(EfT3*&wub}D0wFnVmMLZ z4T?CL1cqHK=&fT$ZMh`4P(3-aWj2HOJUoX`7cUEtWI{T3pOjxBlgMTI-NWWVUt6HH z;#7kP>01#9!_o*{KVx1Khy5T+9QQ`uU%$l&%yh)Cd({ekz-8}Yw)v>V@kDd|xUJ9Q zaEGeGu!(*An2XEw(rdABnRPys&&_Z&Ss?)BX1ES*6mv%jvVDpN*eWfsY;dl0ebOFJ z_J1;d38r=3tu`|3iA#c1wgZH79O3YAAN+6_5rtDX1_}(NzfP;sWVB%iH~k(_S9fG`DQ_w_3nDS|EB_?Quz=n)wH4PS69zmiAD>eSqzMbG2l2iXEI(RQwO9N3?&xZ_YS zwuu($NZt?LNzHnHOk!xf2Fo~D|t;p=GQ ziEQ<~y9M8~!>aAgjmbtMgmfr`2JLqAG?(U122{b-RINU;5g!Ng*|U6jdJOyKG;`F~ z+%_z2k4vJ?rvq3X8*c!d*A;<-==J6#Az!awm1lsb=HO;#o3vn-6=$(V$GbXyS3+2R z7nmlv-%+Yb2w=XL+1txlO%*QdFr;sZNW|aq>c&F_>2QN#<;9p&bZ*_Y>=ZWFHX3z7 zY(chL+eiW>o(P$3+cuIzPTP@h&w-eYT=2nwQfFJQ$dT?Xy{=JjD&eXy#GZ|!eBFmZ zSjOu#iV>ts0fu2Nm8Taro&_>^#Pr3GvoBz`JH7_4PwSBV1jfC9QdTBt-2Ft~HHa}j z5w^NPp2P?eIin>$0CTIFVCe2!<9tt0IxX-zp94bsw?A#@?bVK-JGDbaz3mf(AD*C^ z<+T;{CN$yl(_}?_QnwbI>=f(#p%}lt(X$VlrM%OVtfC?;`p{)0WfQ$CUJCon@`mei zerBb@trc1lcQ$aL8Ecptld)~9+2cj?dSw5c%#ECMbFQ=E1F?7zA@cfj)@Dg@KZkXf z(F06CV$h8k?HS%bGQ*#?Tb#T48tVk{IbZ^wN0{v=;Wb zbv>e|QGLKZmeB?SJt~AcNq<+6Go3AdVUkr3+r-bWwxyUrhF0G>6Cr|mHsIf5IYq@9 zc}5JL)SK{YSiaybnA}>Pm;^C%*O@>n6S%w+4|Mv>j?8l>IO0D28(|dIl7t@=Wf;(M z^kb19ES9T3KT*ZSFdBSm-A$H_PNMIZ`M{*^{QU@_VoS4&z}tEm^7=-VUITsm{l=Ln z++pDv-f5z1*kk2l6g=(%(PV=Em_B5VL@RCp^4gNyhQ-WWgEd3XUrHlI^WGsQY`_6q!_@IKiCwl&h{$2aq zcYxbLYkhShz`F0;$1EHvl+o9e+jXi-hJhnGmSm{UIA%bdKRWeE2Y|7+X9lNsOLn!Fmwp(RYiJ-()nz5E# zR5ecg^M11Igz*R)-Beq82oM7S@bt38%Cb!_{jbPs6MJBCoC!jAU z0CoOKT70HuUy9q9i3ANxA*d(2N^RVM8wz`e*zQaSy3GO@*k0 zcg_6_TtM1>?6m<&zCRLX6REg}Z*D@_}>DbFRLh;rc|hb`cmS7(Z4W%s%d zH82D38h@wC!31{Un*Lb7Z5OqO!+$ zAPCzi(PYX#WxE!FVn4fhGjD3Y!@q7weT_Ts<*j6QZuB)#jSVRWx4CP zX~X_vo(uJRG?ID?G_Mw2?^9HxZ$m9s6VDB^(H##Ay7HdR^Tvn@jCK4Jmr9T%KT^FKdOTF`*IGOAI(} zbGcB?^!!Mtrk_j#Z{+P6Ok|M(M=XH@yiYYw!*{mdZ9 z07|)lty=jCx9#mAcl?Q3l=@B`;=;%S%tsacT?$GHfdX1c%vDxw%wZPkzP4*I>C)nL$uC4}4cdsjpJ zVwxl``Qcp8WKHt;-4=f?`w^*Grg8AONh^xJ{pT8??3x++hmm8J`fMCC{PyMlLUqeBatCaP$=+ySSy!@CERoj1}jZL+7| zq(jbFvxkOi2GoTz@y@{JU&x7a)#fdD)@W~ZJx~Cs7l9olc)nh0ORkf`{XHEboYn4U zooFG{eg~zo-qj%!oXlebREz)IE<@@+tilx^Usm<)t=~n#;tl(5-3D|0RhM-{lMd}nEo3sY86Q>q>3!QM zitTg6ThQ#&J@GeSlVz0C3uHH*a^OOiH)5Cd3Zn6Y*Yo2#tjjf!bCEDB5RTJil=_a* zL&rMZEf}?+tg9Dd;T2Qj4g*Ga>)UYSuFK)Nxy{`8*_S4sk*{$C8+o2P(Si)^X|Teh z{C(b)X@Ol=;zH|IXclUrB4_vfc+P_&iz46S^FPLiFwv>DBF&z8pF(qH|8EkJBc<0V6^ixq1sBm(TILReQ)=Rm|8|j93`57p2WUy18AC%$biGB95Sf2 z9I6<}k#3^_&~Ovwxvks?98IbixJM@w#rUv4plfO4kSjAEPqz^$080SaF zwj2b_yuRXJh3LXiY;CUbiyyy$`wkQ?)pD;B1X1YEk}LG3;WWB5CMjgG~~t+Hn4OJ)_CM1z8&;h#xX zMVKMb7>g3OpsY&N8vS`kla#M(&eG*xPmDb0mm(^ehz5p5&^3A(Hwux&?w`geMYeC~ zwFY9A$e$VPnFcnVKeV%V1}7kD_}+#nhPs^rqV#(GZlq3K6 zb162&+^9VeTEr2NdJDsvx)+*R=^b~dU{q@!TKJjd|jm+M9m1pXo{N978EzajP#B$$mCSlvP7Gg{jbi7PzfY67-75E z0{J(iJ2QdHmbCmV?=`9d zVYG7D{y?!BnZ}F8Bee=q$2{m zGBOf^S;+g_uatM2=jkpbCMaCDJ?C;FJ+WC);&JGd#8PHqEcHkHC`~+U==+E!nA5kv z_;}xUsF8uLA_88M#a@#omT9S9wu@Ume*w67ikBNTpyE%6yI!2U`N*@_n~NcjE6#!T zx$VC{*m8LG(ss>YmWAkjlf!?yS`>G6d2x;VI!Zfm+a6kKs%7v@enE-8VqKtLe7;*2 zFM;ev`b_=&G)Fv2GT$TyRL8vQNnTg=xcjr&Df}R=mbq%xBwWLGpqjZ;*LN`2z{8my z`Ss`6fPw4Gww-H%eJxKUyu|r2Z`0;lg1f9jOo%ZiZST^FZ_u@Szin#8z&<;PUwBjG z1arjc6^(R5E^s5#4cSAUAN?nf*A z=g_uvsQ`iky`N3sSMVVQLp!3!%+Pf!xR4tO1a0=~W}_c>$`6 zjxL!Ufn_TKb;WbZpf%Zg%eB@{3(y!F1xzL(leM8HoN3Ln$`4#?A zzm88wFc1twibtRnW`narGEo18F&#zxem9m${a*}KK8vN;s!fzn1`McB@HDd5!B)*o z&7J%kL57d|ybFHEhk`#5P5FPhBi&S{hSUE)oC=OS+tp_v$6Ij&|7Kj^X7Pf>Z;CxL z+lk5*kN@V0B;zn;zPFw9ei|+p?^B}q6JON*17C}5EQ>)zd*b~seIt*^GBoYPdg-Ir z^*5S&=<|7tK?}Fu_Wv?Nt@;1rtD78uu z0}Y7)C#)*Oc8yPe8ret3@X5q%^Be?FMJEDk+G~QS&KD)*G`9SKI8pH!V!`=_9lHe- zG|n);%)bk`;h-FTdu~;l?BRxG%bwHs>r03v>ie$0n4SK8VR2++Bswdrio3@_YMrZG zi8@`{xZ;X9%uGoMBQ`enKC+u(9IKUmcXzkPs>$9Es~Pg>rO-NUJaBAGvL6UM;1}u8 z0wdkB+uJpq-FLa?HohkXQ0J{TE`wm2CS=nAj(sqA?PgZoYNe-*=5QaV-m0?BhgUbM z^XjpjPy`!4l9CsFDFW$YIj>_mh|U-1`a0fQ%p0q;RzM*H3F0+lH~O+oCH0*tT=j*1e| z(<4?Eh5r=)n7Lf3$qml=D7Wp;bfjr{p`xJVV$f~P_^Vry66Sg1Hu4C&1;^OLt~p6N z=OIbA?eRqgt!w_Q;J(et7iMd99Q%G5>8O%vj5&CLyH}+agdwXI)^$#uYSnko{{8M2GeLieFs$0-2;tedSNJy(TKur4&-Smbtp!hMxwc%Q7EOi* z!a&C278Ow(4*~)=qs6y6XM%G%bWK99#v*o6Q2~=2XSC?TV3tp4u)Y*jPBetX6p6~h z-GRl$i_|}k^_0H~%Z2{DM#urP9SXQDb#9EUS=2&93}%7i|q;u!v?iZEA=1-H6lYf8Wlj}W+*UswNJ+Q%`7#~qE^SpFi$*^Uw zH^80c)9!AB1DrhZcE8-+k*EdZWi+4m?)(}{=ZQ2eYpmrbL-co15o7cmRkMwIzt%9qow#8#qRbeperw;FbGB$?`(;du@hw}eKF-rfi~5p-KUHTo?K zojjvBw|Kqsx<=%Y-WdrS9s|#r87-ZPG=<$dInHn{LR!B+_(8$5;5)MXFt%c%RKrHRT|diRsNI))$|h_)Ukz2uctYe(4>uc;AO} ziSv9WlD-sOZzkjteyU)RfATeNIU0aVosasuKd~3YnO1{iKJ&0$MhyiZWaYZn+e9y0 z;sQArPOc1fW7K>+;=MoAarKLu=UvXI3t`vx@R`IQ%|Tr{sNtrBM$At{;w$v4U6?4W z;@`V5C9GrViv$ZWw?f?Uu!Vlp#Eos+R44F$wVcT{L&hc$H!#3bO>2y?Y}8>) zfr5|NPeo?mRJla2nq2v6@KbnI1l5NTADi{nw$cJsk&X}1LU-@!?0f)mlmRC^nb)e^ z3O~{5#r3$8`(TAhEdc=rSc36#xyI9P9U#E^UB=gi3nNECGkR=xU6&`9l^3Mx(a09i zQpGE^)h%%_*GaO^8rkFT^E=VgwXwiie%6PK)U{m2Y(C%gZvGb@Lp-@HyeLbXYKtkR zurw4Kq09$M8gCGMr&@CuSA_b^)#0}}W^S>UeaKGL43DXK;WtU+8EV2h?oP&qNUX1E zF0h}(B$8i8yV~z2+Jx9W5-U9`btDzgv?m@@GE@Tu#>i~0?AcKd8fy)2lW4?7cf-Uc zbxPLa-*E5n;6IdGJbGckHdgIknKE-)Us4>JSt$j0yxf+xGaKW=<2db&Y(WI0ZkvKu zqwc-N@e12vCN9&$#c=J~?_As|hufIM*Xg5UY#oAtGvanK_kXdG45$kiz@hTWAE0=mY)Z4#%RJdRx~IX5xq9>Qu7T3}_@KN*68j2G;>PiS z{aV_ph2b=PXy5zwA!ujE0Eg8~=;`((YA2#w*FKv)t;~`G&A`y+OBIqpUI{LK;JP<= z8arOhy2<=!Bt_?NeS}!o=wvQGUxU;m$K45AdDpW8$b%K)!0q^oa`i@k{w}wy8btmH z^)TlIDr~>`qQgXXkSvVCPUSa1f55~2?B0YJK-j?0L%@^nY`WINRY5Z~@X+?*fZdNZ z48P^`hc&&RaD_y=m$VJ9rwG@u#5nEa`TmL(E84YIq@vLP&LIJdzDvv*J1=7#m9cH? z(wZsK1_OXfS_8|Lzbn|z-XTA%9H2$8 ziySbf>viPKG|t;`Dzv?a!>3TQb-SoMwE2a0b^*WL1ay&oUuD0W^JGArX#Lg;vPmSI zrKAdrMX{xuB%&ts8xNJv7v8ruUokVaZ|%31QEk0J5Lr5_WfC|pYT|yGA-atr(yl+U z4{>y=8B=|AA!+SYuq&kuWV422)^rB7Ek?NzHgHpg6EWq!)c1j6I0)EPx6H4+ZjXmWBtcwX4e z8=xt=k7b|R2nY%0QgrQnX~My79MVI<5pOr9o7AO|0njM)VEYr*lhe|7hVHF>)~Q3U zcHcc~=Ld;2TB;8i6Ml}xdLo4|{}=*L3I@cqH43(O9UO2P7>0bHeZ7Bju8U<06_^!t z5v76uFd9{4zlhj692mZKj4lE30snaQnpd7ZtbSzLrS>C6FXfa$zsAA)S&!=H_-yrJ zdu$w*DUUZ*bS32d`BD)mwThjt22L)sG_T@6OrHW-PV0NW? zIh-FmLoTvM3swB3URUH&sWvAK3zv)fQF2XInoLYg^;@+wC}irvwcYNUK9Kd*&@e!; zK}xC?GVCckI5Nvrw~Yw$hH8%nEYF{+!GU^`+U;Z~b2SKI`C-fJ+ zR$mwrE`TP9!+ty^v_);!T$=AcFU+dBHDWRvOQlOM+9ZKd`W^bENnR5-xAx|hcPS^i<$g{cHer>8y zj&v|SPCAjs)m&tvomM0nqfiN;X95_) zHX{c<&Yt!@`x#`OVqL}Ed%b?Yc5I!g5`=S$3Go@ANOdOcotw9GaU9F@Xf?!FXRieE zRbxxRVKf?J*6IqQEL>NWa*y=f&Kcb9gyGto3JOFmz*)1HBmN{XE>Oy=%r%u;|GoKT zTz3Us&F>Zh(*9d8MG#LX@oWe@gkvfI?^gHf#rg`beKl)J7WM}TZmDc6L6d|zs|5F2 z&CqXjYNc~9x(5XpcXS&V*6_t_#@FF$oPD;#{Vvaqz)+Xm&sIE!)Nj4P>{n<4wX&Jf z%>`)CJf{CwF%Gb#TR+lY{i!?O)}J>GT61N~Wa-0EeM+wEozCNdrHg=vmItDYC3R~| z9@Ip!$m#W~&{zt1N;_i22*uz;sJPa)Xz?(i6Fztg`Qc`mw0L z1&40XX+1#2Efm!86p19IxAbz9tAS7P@OA$Fexqx= z-d^L6S5>&-H8xTx;sc@m*O1h~ql>Am{Jvygv6Agcr}hGQ^&) zEGs=vWggegp-@F`fg15+3)cLTu-fGV*l6EDxB!Nmy~>?Q&!DGH4QU$z34z#Gpu-k$ zZemPl=`mui^Oa2NR9F9U(e^nLf28_?-fn{|0ujf@i3S101Q&6P9UX{msu{~y#$*%^ zM^Xo77pPq#D}`~+wl}Du?i2MJfL^R+7Na&3;??dpZ}JQ0ZCh;WLb4x6l61n1?4J4| zw>zK|FQt^J2(BN5C*X0@Pq-;i&!5~yl@ZZ5q^hDROJ14S;(8H8-?7^2xZ5CU=M>=4 zRJdPh4p<}ay9#2g7Np(ttV@yN_v#C*)Hg+!tNBVL_L!E1@hVtZ$HuNT z=ljmHMNAHM+F)+W+g&Q)pj_paUfluyx3`>x05yD8nQB)Kg^f2RJ7e0(16OXJI52_& zE7W3!m~`~3HI4@Bq5NctaoZFVvLp9wnZ#U9WxM3uJ#HQklcXp5Xu*%!1iEY5s+DGP zjEfnJf(pHW6BsTP5Bk73R`pw0Y;6BwjiX}q;L#-d{S4>zX`v`IPCFb-)6YpghmY3> z_G5RgcE8~_!peJ*DWrAn;a4`5Yp}F~2={xhFOpH(^jqJcVV(wcRWC8&vV^9iM;m$< zFE5=IRYlqbS+g}QFJhf@ZDvG?>W@`2T|ZZ1gLCc5rWK~6en-0C*gv-TIdbAoW$Ote z8+t@{u~GAKKN80oZ}DtaIzR-O6fN|qqz0=JH^gfn%S1D8HTGs)fXMAM+eXFttfh%I zpq)ofD3ry?90rz0&2>N2Dy+syllE&y4k_*)6yu!fyT!ZU1_7iN2q>h4whBjPjw(J4 zH|9@qncaAs$Zb3N%|{-hYk(0dx!)?ny7tqW&C7%4uKGjBgYNP>pgVqnpQ7c*Bk|Ju zGEzT{wp{?KJi~VSVEQA;bPhT5bTRX|=fvGn>Zvt=Zzub)_-#L;S=sbQHr1N3eBQGs0vQVV##dla=YN?diTBCi$g^ zhPDwOGu+B{2~WKnbwV4FL#7_aczQoIci;U^93{)xVqVaw4G7O zJGP6Ivq0mR!Fp}i!D=N@j3Z33CU=x0V~1;-TE1&T<~zk$j

|u>?M1A{qb6~Li&|TMtRxp4jeDor8Net!p1!>n_D^2J*Ueg8EnIZk5N zuM+u3m+_fa-*w7H-1aTG+x{8zJ@afyiadS+xn5M|XH3U%fuyX7)u{FDvy1@4gdg+;rm}@%JhnNR1+o!Ve@hJ zKYPnW97>4lcbVKUJ1H;5a6LhTJR!@gaKX-A<;-{ z3rZ{aYd-dk6sRac)ijbJu2^CwB>)=x1Dn^;^p0=U4e!lFgM${v<1xPmO(1^+kk3qY_FutIh)m@O3$|`tGbsP6*R7b#%b%3dUD=M%aK7ufwxo~rJ}?Pn zkLW&h*kTi&Wy#@V6#~+aI(Ry0rm{;TvHF%z!Y1Fy-~<54EHtiJyXaTHXFzZdNpIxw zLXM{m#7^0Nj0(pSx>}x|J8)!v5df)NSu;KQYTR0eZ(fYj6?w)V`0v%xBU10z(Kp8B z%n`N_D>afD%b{UM4C@eo;qfm|mxRe!2XqMq*KQ7}^)En?e<2M`p3;#dSQ-WO7&YlJ z`-cuPib9dV=JdQ*D0*Tc+#FeR>O9Lk=q*Dcy@IReY0*oym5YyA;$C1jKeR7@W=#BY z5#5SUW*&4@oW!M>@zc zkYDcv>MF}}e~n@_hB)K9LpHxFLQ!S*y(Z+_o(9%6jLnDOii;tXRN~abLzmaOkD|}} zst18l6H#w^Q6@f;i#7ggMvza96~s@hogf7W*X;wR}r}!=IF7)4}Ut;nJ`FDo%3xq=J z4J{VGmacr2TqtYF5&SLZ_^Sg<&9+?3Wr~o5Dm;?d--pOQ7EL4=`K0xPY4_$x;SFpF zxkhJgsk9!DC%%P6 zLl+yMyWp0oFL_p1(OWUGZI5sl0v+Z4U<4_P49P8UrU7FKQVXOc!sJ*+KKSNTA_X=) z%H3bzBpZ^bJJ}yZ6ew9*Z|}HZZmb;qUWS#=|UHMC-)n`<0oC% zJng^e)AWayVxbIv~f-;6A77(l(3%OiFFupPC(m&j`an|qz_6T;TRDmUovp{ znmCSYI+&|2P^(z}DEEK5i(v2&rN)?eNIe6ix^+$alcg&VOe?t63^=e;vO2BWz`Q-G zbzH-)lHjAb`n;v{l+dXvB_0K`qsEBa>jd&-){saWZ}>A0ZqD|j1(~!%Pmkeb>fn)a z?u|{`nsm-v#C^U*1B@t7n^7G?k_nn3w(hfuN47^|ksYI>n|kJ|PehCTrFwnZ6MCKZ zvtK?fXR2SrJP-qIJmD^CZ;!ea1;id=ZSYj)qGv{e!hauqmT0MOUnqBE&z=%46c~^@ zy3(IiXFPj;YF+}!H@Mxy$Adzc3mk3>vs=o@I&qH`>e21CfFWU^s~vMwJ~*Ay`%N)` z>+8Bd8*seQkRn+WIdm%6tk@J|t(*@Ukv7tFt6V6@TJm`+Y8Rh-I%9a+(6ku3B$!2; z2?pN~F)Y;pw__n{{4~M42%!hX7<_#quBR>izA+ju*6E}NT_uhoBWz4_WidP@d zVhzk&>RL=ecV{g_Oxvd7o{jRf0puo{ZhzRrv{)t=iZdDx&0dMPyX;0^3+51rAl9Zj zfcrz<#^r#tVDt8pYU*^nSOZ7EbANwYm4@<+jrF z-&%S)=ecI6RKeUs90)=G4Aa&?Id@Agqba96<{kTE>ar*JS z%krU#!%+VXbYdVn`I{b)}wS26Z2WcqV3-G?I}#ZL*1Qyl+xRHckO<8J_k#G zmYS4tI4yU6rbiPt*|%Iu_@0}O>~d?NI`TMFcN-Up3(hzXYRpT?oX#3irsL}r=FPM) z%rx5(M6-UNvtxvsz`3W`rnV;I$pcQ}3lYX`p^!83o_S?}h!KjN7sPQzgi8N~1i5E$ zr4zEn8r{bfBn$H&w$337EJ@rN3jLNVJ(%7-Yu2c;z#Nbj4RM8x?XILs_E38nbuI4OO*Eo}{fg6!^%SD| zccBKw@`#>dbOAH9MB_e=I4G#mBaLW8W-Il*yA)sQWasP9t5%RO1U-1!N1LiJy3EYi zTsS!iIM>AlLu+ohnJJNkwToB2Ipx552D1$J&yMIBiIq`faknCV`^^4Hm@NnS!$@le ze1gFF94?E0u<0G6Q+f()U_`FOb4UjyllF_}fI-I>$H3##u&Ai2LWwEC%Sg2HEvetr zlXn^Yv}6uJI2!G`^kW;-Qc!UWyz*+5#y zS+Sn0`#L_u<>qcd#T3_C(4|LGSpWy+6akyO3CC7q9`nkGIXxZSya00h^^AmC4~e?f zf~NDD?S?j(ivp&oDATQU?2zMcdAXtwwZi_rAdcWK z@z@>e%)m$eaki+cH;EovmnG!qQx|%IBVj0cM3078{o*5kD48pNYZO2ryq-Lx2s_2V z^Z<(iXIAY3n_nENJ(J?e#}{M3}nyPJDGlt;MO}?tp{VznkMaT35iEhml)#US%dNpb=V$bExg>^L#G)VuIwLMZOR2nHGs2|WT1l-1jL;c54ZnFd1qjj~`JipNmUqYZmofqzsK&d$ z3#2FUaHrj=`Nt(UCGPK$g*e@rHSTE*!~0F_S0xc{#}tQXU?%p4@tjFC{AyI7A3b_#DHjVI^|w0B(4^2ZazcUI^S7T_3ezrAa@J?o8g%8h;D zUaEGcpWuS_4-7ClrR9ps&O_#oohm%KuCt+YmR7xyI`|WjK=zU6;5dxtHU(~A8!qwEKv_G~&!B$kd zjeH<%F`m%rB>&r-UjC5r(ToH=Z>dglZ8tdfj%vAF&uOBmrA2VoaYt})aDe`Ny~D>e zlzNO(Ze~)$(zj5mc;qjwQ!X3d6u}zxrL^=mqu?kiA9{P9%dnJh0=bLd&?%s==w$PS z=Oe#+mCJOhJj>P_L!#ONe!<|ZVw@2>-aO$C<$A)a;v=HFAL2$lG&m(ir>~+C&2Wx- zoC3P!qxF>Xg(gI@`Ef2_B5K6tf!M}mWxmJI8&xASZSGJ2-SuPn?(-`r;qx^nWo{17 zxp!QPLRZRxwbbsNB^&WdW9~ek+;Cd&fz2Rlc=*yVZdJ!9Kerup#e(m5(i$dBsL~bd zBI}^bWY5p)Y#Wm#_8s3Y<%|!;>l6kv|hP(|mkk;^nHZENa>QeG-sQenwn36~M?AQ&^*`f|wtZsQ7aNMKW4eWd_QidhUx7lv`6n zbi1$Ks@qwx<#@ud2{{ErcKMV1Bb@|3X_lcB5A-MF;^QNQ1eeQ_uaucPLOnBMWL=)Q zBXpnUIVm#hB^^x@U4U}IMW8L&?E@9LZLu7Vc89t#cR~UWDcp-aoF~3WB79Ed4&%Uf zgK+()Zj|ZEtXtm9NO)<}kJowVGDXgw@uT6mmD{`PaVmEDkVNmo0mrX|i>>Jwi0i|h z1caBr1TpdK%sK1)&=LkoGNlU@k6buTt=gDbMfaes6{)uJWho{;p|o|6juXk&Z<+(m zG@SwcVN(=Tv($t^ZS?(~UJweTjrs-tSpc4i{Sv40uFi?Hn=E~cEH}V+u9Nh^#c7ZV z9+>yV+2P*$Xtma=?M;>kUm{v^W7P@WBNGiKK)@CwZ*aaDo`vs1#Oad=2*<|$KoxB; zGG(HPPjs{`+-1iBC8t4pF~+fB`RZI^7CJ>Wof@t-CCy1J)3|XcRDe?ouik(D19SXC6Fg z7)FojZ%L-CNZTI9&_{>6l0oQ>lqh@jP)9*NY!5d}*$J&(d=bv@*xQZPKcJzTEd|da zufcO6_)a_1xS%q-&7oqdk=827ge^UT_QDb5g#yt8!zQ%S+L}Y|fvwImb>&}=f|lN- zgXM=V2~nDVx*O@AX*od)Esd^8G7wT(+P2}!0PG~&owglrm*K(J8j|KM5TMm4<;bQp z4a4&h~nFR1EZwIWnx4Skg)O79J(Eb03zUr@F_VRdUw! z*7U}vSTNgEdAR`@Ql#JTjsncjFB-iq1dRq>pjP4v(%PINj)z_erwzr-#XFZztCXHf z(n4jLY8f6mF!_DW0y(l2;5v$>(3;vqK7|EMMFcJ5c@0?bag$JyD*pO&TkySoLX}{aiNBBKw-2gu z@dc0+!Nq?6^_au451gIO{;Pf9c9Ijp(@r9OhmG=o2gD}dqx%d-xz2PA?_Pf%xmExf zRMN=&6=lNhVh2@91PrOx0smA^r{Gh|L-}9-94YA-IWP!qa_GLrwYy^DmPLPNBw$;? zM(v&lpKd}v0ea!7KbHOfX@Q=Okyr!@xys-BU)OZD3djQm2_=#@Eo+9O&-?$L=WrYW z<0dZ^GCqxL=&lA;rdopc>i?!miwB=HkxDf;lDxP1|3qNotnf7;Fl{dq%=y2O8%4U- zXMi!|e}|QhE+fLVJh7px0j8<=EB8cbQJ<9pRq!+Z0CnE?m49M^2WgubS+Ft@M6clU zU!UN5kb&&D<%U{m~mNAvR*1TL@T^&a+T|0!Gf)A1Eo!$={t+kfCkSF`R z6_vdkUO|I*68qj0Fk;h+;)$o528je;!2$SXOF)Gr-}?g2aM&CL&ffc!FX@*sVN1;< z5h!I+`5S)f`*PNQ2Q2b6g<(1&3^PNfKOP1nqp{N=gWPN6?beHDeb*4oL0nNS8I;p{ zFdV8#9RB}L>&$g9v{#d0l8T0w<@7RB5=k-16U@$bai z5?{nS_8dgI5qg)dY>>Pu+W#)Sy}%2YzdX|`_eulhUz)VdcC1)wD-nd$u%3%9eQD%q+3-N;S;tB5|UR#<90_pc-xTg3{KIlcy&U@C`y zs&643X4RGl9jlsn&b+w)&m2=mO{XpQ?XjvRY*#&@%ws3_#U6P7Pj!$HK#zjp>yY_m zM$Y~Umz0J{&fW3n$RZjOc3o&(P~(6m3|L2VRmg#AfUM znQq-fm1kyYI#x3iUy!Y-VJS^^y`ci!{|DJXCcjm1HVY_CVCpMLUbp9nMl8x%pyfw~ zc__c{I&n;~%p;b04qoOK`qp3&`-y#A?y8>_z0sgONL7hj#o<~nI_-JYiOfG~R#UwJ z`Pg`5Lqkocs8A>rXTBCVvaJz4I=&-F6$>leRN*}Q_hmsKfVL4}jFWnb~wQ%~^apG&aHs$SII)@=(JJ#`wtY|Ft} z?-KrR+`HV~CQhr^vvw_G{JoFy%lbVSv@RzPp>?9Lm}+W5L7qoiUJfS$bjH%{N-btu z*A>2QEQ5CJa`CA54h+!w3jGx*AZz7 zBr+jb+#_AZysLCwWqDR1?{x<)t2P~aZ(i#v=kp;R2LJn{uflrlpJx@0<2zS%&w)ar zP@L~eVBaF)mB4b|$FTVI9ojZ5trI%i!PoF+oSDndhreD8)OnrSS?b@|t z(&QJ&&&#jEx}a6lG)!#6kRC3UXY3(Y_TboxNM5jj+HpqKrR`>Ck+*V6+w>1j+SQ|5 zk9bj;*P#EPzI2QWL+7%Rxn(0ieffGo~v2FigJr_XyU_4e$D~1Y;N4LW){m__F@j?i}h|PuykbtCogpu z8`o`Ocdosxf_Ijbkds{^j_D?6?{>B&Ws;MdPkw<-J636DBlf%ka`H-W>B_qrWml!M zAeX#ix7OX2xobNcH|`+EVbJ`({6&f_Bb6=dHnL-P1|Ge4z1-iLyO-4qK4M*4XfS={zn9vks6~WY>J!G@&K_bK3IB&djAm zyRqJMyPOVkGqcIJJIaLV!Rhc?*j$A?zOFl7c+4tyB=h6$G@-McU4-Y+S#E zJ=xy-=r7XT#Ru56S>(yKWbHn9*X!hg#eFT-8ViN{lG(6!GwJ#6YK5mlp-`MiPK!8C zva&xC>yy93xYliV4umC;`SRmWKN1)a$mr3dF&d5N#ASat?IR*1A{qPeSRq^jdF;{0 zxOw=^{4(bkh7KJncC2(9QG*(4CJ^7YH4`^%BeM`9oqL(HtcZ}%Kvpi_MbeI3TC}mS z_3zEp4(&syXanY?A9&!|w}~I|Cc_L181vc}gf)+0@${z&u^LIgp{3X#l9qkIt+#(h zR$2ii;ugg>?9CHX9-@iaiO#15x=q z#hYMs7qRxMM;ZHT5fLGJ9Q*cD>~IrVry=)Fm_ql203j$G{QLS-yzNn3+)Ig6c-dw=nkT3?~%Ota8JTq@>wrr{^oH>9mu$RqzZp4JmzVa(+h5r~Vu|9-^{_y5REW1ryO!7W(u>}XyW=_ss}j<`S@ zo4hjfloHk>Y4|6|KnE~P!pED_z2%+ zIta8Ek+F6yzZXZ+?}odWbah7(ySCu3*$Y{?=t54UaX(?EKhwh zTZpMxp1ywsiMC=wnzZNPDfiH!MFh^he{#$J-lU{vJDzxR29s{?O331uxaq4ESUs}) ztc3J^c{J}noGGurz>w%1e)x7OPydl7F5biTPo8G-*YjD{`F7s>@Ez{D>^v5|{wNcE z_#3-Od-BrHxb>OOh-!BQlRuiygsbY4`s>?_n*1xyQp=KfNrV75W8L@*BgcJ6M7OT= zy7YW;c()KVUdv>idhs+%$mZ!e+t^A3UK zcBSd~i(-!nw|FLZrlsL38+hQsS+bM8$p^4|?M1!^0ewj}f4}z(kIyTi`z_;`_TCf* z#b@y2TTk)&CJzC*|1#^lZFl)|tWQgQi91gZ^TF-$z7dKCw z%GAe)5mLB^@vqHczr&-fPYQ+NthPLz@IJP)t=rD&bdtO$nYdbU6-{GsrlPTP@g$*GF8uV!46=pgE zX0yfH?Gh48mzHg5TI?XAK_lA524d2=DJpPt-euSF=>M*#d15oJ9&tYn%w~!`LVRTSqhI!lu;fcsY|sbRxiH^gayYd`6Q6v%i0fxTmH7izbxaLfm~d z<#%eeYrz!*T4VNvaqX=GF_}HgUbhd65Oc-4^LX``50oRUoVr>auFJ8lU zmg`vhmuqgM zafplPUV~}bycwx;){y6NQJiO|(@i%sdgK)}Y21zxQ(mNBei~WxSBi7yGHu~z{*Jqq zg^vuOUP66_j2grJSGQx@%w_C4P{^9E-y^6)Z~kxGC|cF7MVAq;GQLA7xobXRgSa@e zXk3`>h3sCtnD?K0j=T=Tcwo%KGz&3_#)hY~(IH;O+D+-u`%1#U|4Cd+4mxYP5OK-$ z>(GI#>lHC|{st`_e?KxR4q+`+83 zHgnlcBY5J*Zp77X!QD?zrc2uv!xiT>r0K;saMSs1sIzq;J8fAb!XJF<0IqX}qkK+1Q z6B`yoV(oSmFI>zwp-zI0VO-ccuF9jhT@J4tDzor;gd{Q(XbLA(t`U{7JvfSvqH&PGz;lUnFaJmSmO(R&L`H?yBNNZoWmc$2`wC+=nK9c&in$V;uQ{=x(yC3>O1I|zv z4u?pw(L+(D6$+03=?UVU1NG94STo1{Z%%_ zB1yrf5Nf1-$@sfxutDFJf%n|RrFBer98P>5-#U-XhhX}&@62U2r}NG3y>zv&=PN@4 zZtXl+yr_BDZFX-uWDGEQsph!SbQF6nzH)S-5NckxrRoDpY5q~pA~GZ5X^_y|>nJbo zgDds|K6~o|EnnT9B0Ppd=Dsn3HQNg)F%`08@ivV8FQ>%rJ%&-7D>l-k-6xL_-OcNn z2?=y!@*L{y3WY*(0_x`fxZCcCe4hM+WNHFQ6ttCnp3cTl1+pSHn@lus){NC_R?~k# z|FS@rY5UR%4h*Jw%kpYs-MV$OZr!^2d=#6y;}U67uQk(W&R|)vg_iep(hPt-+P0Ff1wvsIxaB#;b~YytPcbE8O{C77k8UASJszjlT`L{^^P?|d*aLmo z_3`U0Sp7XqvzpPRhM__?x6HD@j@@Y6r3W8=@(t6X*3i6e1N!uj69R5aSvt!1aFwM) zp2+Darq~+nb-1^P{L>jR8>=r}JVFKOtR;R81@W=F@R$wcF8Gm0MvNr$qTvi5*pKLd zBH9Ec^Nh_=X2p`voI?sgN=3R$sV0R&af<5aGojn=Xo6&(ZJ`Q`W)p)34Px%xxvczm zMOi3T`B zHL-+-^Xs0>GIyyx$$QzfUv}k*bDN`R9+yLLP%s@YzeeQEUgIu47Q!`a&e>F`IK)k-UrD zB|2?%p7X#qX702Q;t7y)hME#fw>g1~-=;+58zag8nU0d0aTAv3V{2zDTJcL05FD9yBKPRE5 zU!hPaj>+=$pK1edi}&aSkI(XSwudT^H|T;3E}-WnJ+;7?PdxF2<`{nKZMSK|_8krf zix)3u(({wJ>E@eg+O+8*^3!X|Y1uZG$Y>qA>-OfnD1)X8Y|)Kr5mSe7cM3haG}J5_ zT10VaQLx)_l+NoB8&#Y34YT>_lQ#)AJj5NHYhZV~Ds_K($P?*uSu0nXdZHS2V(f*_ z^Ttc#2ramtM8hhkeX@}K_P^TH8pHAcJ zUA*_e2i!fPkn8&Oq-6apeq3+k-uFJIvDikQ+kYAow>)nB*0k%;i62&e&#pmN(=N0s zYo`aRI}f{jru=hF9M9%1w~xT5@*LCmm$ZZcZd~i$B7Jtjkg>ELic`*qQ!9)&u zh>N~?nCBiJMN0qf6wRB>>=X~J#@<4LcpY}wWu6p}S74)2uiJR>?&aM6@|#Sr(~_}6 z&KGjTXHAmR@rdW+vH{fnVKIBs4D{;XMZBa!fEsEtY-nHp*dWqz`&e!s)Pj`7-|$Ik zI^sf1rbjsw(2!Ln+sCgY^!=6}*=^@rr!8}rE|-;`&TBrC0!rE#d={yK{CpZW zZhYEZ@Ydvg)Qhb_fZ23pRu6HV()!e)eFt)LbD1@37K;`wV(Yf8EL*;e&p-cMvr64| z%WVu9Jh;q9>JX}v$RJp`C6$Jk-^f+XqqQy=_5Y9C>4WeO)K@(^Cf;5({ zPonmPS8_poIF`t83fwkUE?dPvBr>>bV~W#PkrmvQUKh61{7rCM*|Kmgo>mue#rd)2 z>5<_QLSkrh{(0DvS2BOqDt4xNaM+7+21U~2@qfNq2?Ow{JJ1 ziybt&yDh}n+9^I*7kljE|(nLP*kGzOJJ)05` z-<)nO<5{zMG3(Z>VSP>zmyCLf3D4-=NEA7U^N<_U*(p>qVR~ncsh3ianqPH;ft2LzmPAIg&E*e7ZIY z<)7sX*|=^!>q}}fGc@ zed$=V^e;B8-@t~F7?F-=yy+182FBE-g%IT{|NNVco3|3xbs&9X?YP6*(X($mf=j!0 zwW)$cKGkj6k#^={mi+cNtG4YD2RNUnCOylQ^}+~j(12RSIc#3FmQDNf3Fv0tWi;JBum&;Ju0ww?UeI(Mt;*0SleV&|7ht@^sa=Xfw+r>80 zsa0B^Qd3gs*T4U1cRznvxQ6a+8xtNJcxaJR3EzakTQ+TG&b+x=RPmtTAX>I;N$=jh zNk~Y*<0mo?Dadq&(k?L}K%M@P+Dju-OUHbl-qCx5G5RL1_2|441?!#S9!a~Q>?SR> z_PNWfPr9;b(QfC#lLc#0qvblg9i7EY`VZq6@^KPRz5hAAYq-Qc*K0z?W%pVXeK9{Z z2kz2@`GP5W9PTooDQW%EB6LgNnX-wDy=gI6Gx*@KyO_AJHQ&y8l6KLSD$`bWTxYrU z$+yh6y{mFKPKOqxQfzN?>G5wL_~(y*5h1T#Zg02C5tUwtdGGnW>!g)RI>{US z+Zr;>I`3r1{?BUJbvj+BC{GH-xs6ZU9Lk0}s#UC_R(CtAco3?n;B;b6lc{#es@l9OLZiM#BEYv^9h(nFl#i8{d9=#g8}QTPkuG#;bZrM%hIr!db=QJ4)T3`9TdZZti^B{FjKyfn5=6}MQ&(G!;XA{02 z*s|JtS6w=M_r;{I`o2~5oXG8sCiz@>s$8ejgk6W=i%2QkW6OhdER@^GZ*INW-j zknM5=oINv{Bu69z$GyQF1KJR*b02qbdR?XXP)EA{c&_PK&v``GBSnhS&c39*=sOMM zxgo>3BsN$RNJo2~qkT3Gwf*rvX9|TvaZ>8Q|Jboo>$VdGq@l8Cs;b-0D~`Oeo}`#^ zI-Cb}+qsO{ZuN{R6pAxf+?=d#yF-bmQ`T*FETiqtl^~ht0xpmDo(Tng!n<8GpC3~I zK><1?1`eN7n1TW^>%84_O4}<=ZDps|!l`*qLOGyNoNXMH&y&k_2tS&$Fi7UPg6sxz zpTwHFN*^CZl>*A+Qhr0{GQPnPm7fd$!3!rxIO)(itTF~GX0sWCiZ!7)?dz>GAi}uM z=cy`(@v;k%4&x5h9ivbv6vs+_@>aI(%f)8%20E1^AwwFUPwVI)PG;~2S;o7QmN9$1 zlRkYfAud#>ti+1byQ%pf3;bj5w)0VWw4FksP$-U}C5@Zizs}(OwFcUCOvGq&;Bh&L zt=*osGNgpZh0|>)i~KJ~k{bL>L~>-VORU!)qCY`~mT=0D7*$j}u7Z7ROm@;$j;z!h z%0iTso(#7~&ZgjhJ~{A1ubt07WQ9U;E^tJar$bSBw4FksP$-V6EsaN%#fhzYanH>c zmoLrP$d|J=QLpc{bPG-8*B|ENY&U?Ojm#|n;X9V^&Lz11d0g46D>Y3S{5ktiQZZ4S znodybZgj0#!lI??aYWRo=arYz;J|i%UAqfM_I^@wiU@1ck*j*1ht6QeV$osH%h+Ef ztoZeN7H`>4bgOP$anbpNSc+Ny+dTf>oPs5`377ThN!`#Kew)3P`u+RR(CkJTs}!fK zv2i+d+Z`$XK2OSFT%k}Xj!`C6<724T8L;MOusLNf=He1OmPqPHMq}KwkVW%r5L!En zua~BCYd15Q^FH9mH6?WE(w6OuXY)Z|4esk~WZ{Cv6x2JP3+tL$Id2-v4T*H`7|Yf_ ze`8)~1MY2^#m3)0XIJfu>6PH-$8WzwACkcJp)wx6998Wi>8JPjcCDTB+SO<0!dc7= zkLDWJKYYC?iPjgM$AM+@c-<7kqnF08e|HMec*C41?WEA2Zy5rxRM{Nvsqhy-`YP0r zZ~rPs;#FGc`&reH2mfskC%Sz5%YCZb_Ncn;4(0PyzKs%23WY*(P)k{-qo}x;!lFX# zcKbmB${?rE7Uk!G^woSlJxPcWJ0+HujJoe?F1vL&?@#`SSM5>sy6INhCxqfMT+Y=A zdg{gr!IiX(MF(=Rc_MMuh^FKR9wybaZ#5Bn6|EE-oM-EUOx+$$4G3`j$D*b zpZ(@&EaEfdzG2usHg=|@duL)$oHNh!k^X%w--lavr28CCTHMv_TPB_k7bNqDl%`Wn z$e>UhS*y$&{anQvJBQHg$S)|MsHljrurNYGLaJ=E~aXgU@jDBoy1)opjGDqww?v#4+0JH;_j zI_I6NxDJ|voWAq=jOcFF&^hG=ywzr-xVV_8s3-yg0;+qfgeKaz_IH-1nrRW3$?t!zro}ah{4sY6!IzIJQ;UA{@(b!9d#y(f?KVka#qCTUw1rl95o7(7~ch1(KMoW+H+{Z)**v>~RVJPzoC z$XYZPSEDO( zv$12#d|rPq36sNt!|kC~`xb2bW&vUSMsi>66rO%#7N0j8MboBXtY7jK?{CwSw0aNr z&P6z#f!GQQvAVnw>+K~a*c{#sgMI{d0`al9r`}q2c57`yvWb-B`y&vd64cqS& z7F%%};<)Ux4!rx-+q{jy8eq04y`?6lIRWkYIM(wX()4|KWj ze$ES2_gZmoSe?qm)A2+YuQ1w9S)PvR<{wS>jZWmZrD!kH-g=EM7Hsy4rLcPR8hkgS z2er`zkK5zx8uFAysjlu^zAhwu+lq?Tke|lNP5X&&+ltr#ec3)olHRfhx{CSNs@h+h zr`J<)o2t?%ThdkS?^{>Z@nl0=&ada)%~RT4QQl>y(~ENz>|@E#UsCY@?41RC)z$z1 zpSNSZv7loN7$MzKDj_J?0V;?sHlPR!0=9xhs2HH2Vt^nZA}Nh@Z!lso80)-!@8^Hc z=iVhD`t|4c`~81+d`z}`Kc5rt6Ytk~XZzb3SX-k$U%CB18qo9&Oc8YgCny&z{5KRKWxt9`$c2(kSg?rE#6N zCaTK6ej&LmLh5MH{{f!tP+S)0NxWx?jUwT`XQ)-TA>qc_jC?Yhg2-yrxos%XDW@qi zgwo@hL1<0k1PgF>|AP-;sV>%~*G#V)?!p&bO$F#t?QY?*7G|P~_fw>)N1ra8sT*pb zhD|zTPh3Zy?pe{FaUqm9Oa~)`}=s;}4YSaq7nMY%rk#?>KU$YK0N{lDM z(mNr~d6~yUI3Cmy$A-9?SuQ@hZvFUzY|*qfxJ*w?VJ>M;87A zKVK;k(rOC$arRX1Us=Sv?Ct-nq%QMc1Zky}A$G+{AGX70s#Z~^bXx{YBUzDrbW7OTC}2XKAa944v%QpY{9J8sKHfbgnC5=;d4203EpiI2fRtVFLNP^o?T-~_Zt7W)VcBra4u*j`d;Yv_dUq0_s?I| zmi@ti&~HNp%T$H6b*Qs#)=j5bLRaLP#Lylb*QR^-IsfhhGn*kaT!E z&iKBxkM?nN(M%p2`wFw>%nsC1pI~j7j6>_Qv1`$%Oqw{64`$Bi_jET6TQ?(4XJg~H z@A1yE{cKqL39n6jpIzQay0wYtxcGd`%pElD-IW-#hOA9r@c45JseSD=)VAUe%I7-G zCyz~HbH-toe=vq|AFab4nn?R5u@oo&#?03~!F+XJY6|e<-1RNDO zTE#~T;Nd0jz-pcwIfnN?n#Ip4kl4N@b&K}!&WE#E`P*tfnEnx#4mSuUruR!}1fVHC zwv`W_eVU0gKWEvRU4%66Ov4x*%inm6_kK&|?7@v3$_u4+*JgyNb|%um*T>oAvl#v4 z<4jnQL~hyve*8X#mIK<5z4TK)TfCdP-P#i_o=;vdjS*iSuqncQfmN5F`MeJGg9q;_Gfrd51`1YeOaMo{6qZkW$TbJ+`~LYJ?qyHl%3(Ip#;QQvxE>tWVgmc>@lIgD<}Pf>oS+E82{h%;&E?P0qqs z8TZRM0ZLBt*%x1u(&AnLcBXN!vYnS+d!3ZOdc`cpzU!~! z(^sEV>v&;Fs;WZtOVKnGpW4RwPqt8f$Qyk3@km0p&ftOhTX9J_$f;fY`u$d}==LD* zym%{V^WR|d@A=fJUz20o*Rpy~x?0b+??2^4NECI$q`g%Mvvzaf@bCP%H=j>HP4kOzaiGtQTHnX_gy~uIGgbuX4K6aAtn|4lU9I zAbEQ^ZnFRxJD2jy78^I-{uo!*Hu-Y{3Q(-KCG*C}(QMXU$&_htGdLlg7oL5Qb;sOv zz5NNUtsO$GwzqQckZy!}CF4Og{4Ypa$G8{X$3F0BzJLA}lmlzoxo4XibtkttpQG8i z0l}mITbVoXWoMwr%bwYjnDpaL?w<5MAB?z`bsxUMjGbPLMhEBCEN1KJn%qBhIE|EZ zByUO~qgXJ@?9GgM;&p62p5?30C)42Y=Zv1Wk^*}nOD8-h`hGR@zxs>;)lPBO)NeRf z>`|qZRpp;y2LGo&Qd$%gOViegM1~mspy+R^yQ)-`|0T)OUk;L4CJpi1Je2^bDx0Uv zEBOWacw8P#29xTAV=FGcyal7zpfkxxzGZ;bCor%skTyXhZn(Za&oF=NO;{-Ao1gZx zrdC(_xii?2XCU60#h#y6QrQ1yhL4!QP){MLY3DH3t%EbgO_o7J$E#aY?Wa6Cc4|xQ z;6lDRc&N0Fo%#MccCT1Z(T(*8*7&88=4vgN`OXXExe7Uu4MCA{4Lv=O7yBB@6>I@k( znE|yGOp$$AvgR|gY<3z+iC4J|BvpV?eQ1*OEiRqI z8^Xr=-Dz7lR#lZqK%^6MnUT7S-x^=VR}bAyQ38BFx-Y zPs{#e=Ln9f#fn9HNx!uN2CJS^=W;oireW}tuh8zSonSpxdCvYRhVT!Zouw;DI!siN zysA`{zYd=M4$qsCPz6=M=`v^~_|QFf-_5DiR1O?ENWXsl=+U$1lLlY>wRGF6c1VgW4Ga!V9V>qV~wX26k<0hhMJeuPT)JOGp8y;ylt(PhbF+`IIcd*G1 zOmtS-U}UERZ5AWZxoPZQzm}XZ3pP)E`drxwgUgP~8iU1CGACv278PU)1l67>ZD0(d zK5O+FVxHs`L(H92k$8#&V6P1J`NURi5)?vhlmAnfN&|n8HaWx#k5eJUY~sl7{cO}_ zqc`X=r{f? zvgZE4)CnV*Db`c3p^wqRpjZ1|RjNu=sVY^a9C-R8gK?j?oM&C72;)_1yFXtfJv(>q z%#4{c$;;29hG4GMqO1L(%erK%L9w-oub#%b!#laH=QRZDG-yvAVa@|@6ZFp-t{y2^sRmLobf)xd(|VkZV`{Ibf_#=h8~ih z&e9&@!gXlHTG3$?{q^9tcRRiM6Bh10!JD6bNv~K9o|I&=q!dh6C=iU!QF&}~9ck%! z%~f>U^d4hh%ER63McRjHI8VSS?eW}x@Kh#k6M#bWGc(V{(JY0ob;P&6GDISM+f=(e zpZ_(3PAqk==K_*b4TbqO3e)qTQG@_@1)Ngc>USQeova3}cx=ohuCJk20hi8bBrJ0U zo??$`oLjkshKC9evz#*yC#^A|I0`7vIZLEQd!f63>EC*fi)_(erc*<8lTv{O(z*Rm zhJ*AX4fR6B{FVF8VJo70@B4Y_!9E0OJgU*S!DPm?ZAGM#ZDPz9%qUP*s!CO&=S&FfpImWhMe_-_VpRr2Ed6x!V-77F^Qkl1KIVX#)bV)2<$3C@=ceiF$kTM!| z3Sk)= z!b(UZ(2$Io<54u2gQJK}J;9Q%eqzt>Yk1=GO{C?!{GN9LD7oEbaj(1{2hMUJ-|cdW zK8ejIfMjU)CVu$jEtYQE&60^@*(HG5O^LOr)4U~v!q+l)?zikdc9LJ`y~oot_M!yq zaf>l|D$SR~7RKg%iLI+;%=j5>-t{}vr%&Tu>s7RAC0&$U;ytIBpNjiT5r8MWL2Kf+ zZD;E1^Vz?59WTw=NM@1$x#);k%o{hcV9{@4uD;`muMeZR9Rj$9(63=@g1`8Pc^h_f zYVTS;cxwV*9}!^8h#0S@Z2W?5#V$cN(Qj+}zWD5C_)xTUBL6h2COpf`@3&B>ar5gN zQy4$-E3(X@0*@J^JrZTdE5O75LLlT<}wzGRPP8zTfKEB+EmZr=bsj^Z1oQEf?D#tVER4d%;X>0&+*X4 z^lV;7P00|R*pe&i7P5Nb4=i1`gR|bo4D8#Px`_=i+s^Xq$`$N8ZQ#c1yJB%r=-;+Jd1sE3AJdvEn?{J|-JIQd2xIH6Tvb0pjLCv_>n7HpXu{)951@LvxKgSI zkFS_xdyi4O`!zHO5=`8l#qOhpBwl$X4eQsVL47Uj7tG~X0p_mi+X17w8r}PJgtI5e zb_dg{Q%gd{$}BjO%rT4TSDQFdr>W)-4 zX9d%(Q&U1DO$*O(Al*)4rz>etJDx!;<2bzd8@~T_HCv0LY16Maof_1mQ-?T`)_==_ zpH{N9s22CW@&rBNBderN|KcHkA#XcHb@lg_p0m<Ce9NcLFnw&1Mr9Xhl~!r(Mb4C7-$7epQCt>|W{NB@P0QxS`8`OH_F>ZjX$C zBTL%p(_GuX|9{*;ws`$+y0=UqEXeX-om)(e3srt1#XPC;odTbik2ZecXZ`Vr&RYB)0&h?!n|IM z%<7`XzVTit1iOsAR8giINW)S#_LBN9I;Wf8=EjLhO#0ztt}sirn&yJzRLaw{+#Gte z{wVYWVnmgsp(*L}g~p|(Z7G*Nro!h{<*)Bmu^965^Tqk-A|@uLOad#;$^vo8R2>4u zGyC`NSI<%@jjqajSXKU;B|w!wyIA7jOMQ2#cmB$If29wXYFl4fhxK!=4z4o(>(1z41G;8o4F zEQ!rvvp%U@-=8a>{Ck-Daef&YVyM?H-7`ck+5D%w|-|5o7izfvxk z82Ef%P-ZZ$p2OuBK;=*cRplRCxEDwla`9*VwpQm!FD{j8rE;2*i_~$s^;MQg`GTY9 z^TbrHN$f%Unqg{W%FCZZkL{` zzeKrP9Of8z4%V0ivI$qN}@}m^J~xZ@>bt3_sqp;t=jBr%CCFhe?gLU3AA+; z#;0a@khaH4*)xjIU>--ZEwDRZU zWyN`t2IEpaSK;MKK&vEFK~hU&RV^#LMyByGgYx zSTsHy7zwUclf;BtnB+T(|G5%=rl_U@0Z6E(Zqlj_ENZGN6=akFR{!_9GRdw|-ps)d z4s-!YvYMGAkpEJv#@ryM`Bix8DGL|~Ap%iAG#o^SbCJ~C( z57u->D?w4g_;PY^dbR2WL&7qH)k;Y4A7`reI_(m+1m4rAdr)sQ_~%A|PrsFirVOA4 zK;3z2yFb0r0|>n;w_R2FOT}9S3@!?ws=(y`SP>w@p2mSEeGde9pbSl@yl)iww@X(~G8`8D(ic?zaXL z(ZfW}cY8=*kxS8OJNYy2pxxa~3AMQ>*tUz*k*OHkg`sJ#$GFKs7PV;bYhOb4C3E&w z0T?pG-sls8xlb@n1e4YZrtZkgq~OiXWE@d2T`%5^5X^5~25E_QVqfY`v{iGyZKB{` zS^=KceDgMwzurT$Rvoz__6*CmW)fRx6kq=O5Y?qKkJ~PQtNO@%axm_sE zV$yT%n8G574%3iz`Xsq-0ZgK!2{P$03b2%uc8m<0LP)rvpT$aq#i&ZxjCvjR^fXT9 z7vZsnQ@?r)CY=Bc4hIEx2a4W=UVvyB*+05^4Z@59^aSF*nN12w3+6Cs+5sN=@e2WJ zJjjrLjzim7FnKu#%*h-~4xvl8I3oIo(kHPsd)Llo_$xosq1OPSk8b9-EGJE#e2;mL z^}r|w=DP@Z(kMA(f3=^iA}79e#n94L#M*o_ovb0Bz{-kye=4?pVoqvW(bg2}Qqts%p5p7`i8y)==TQMV z6RZgG;myjzQ9qW5#*vt$mg6G$P_a89OVB{tn+A)Ef>**C7rt04I;jaM62Ml_Un9fz z$n_w_TxOjLa4!oy`Tj^uE0dR2MHv6*idI`rB$G zA`?(*Y=0>mri+Mi%Z$mr6^-LdZlooW;_?7Wy(w$?!&qSd1pTXw$p3cAg2BzXuQ!ro ztx4x^?xVgzhpX!rHWh9Yw{as4Zt9Kwz}L7NUQ5qM6DhuLHA{MKVE55p+;;!9G~Z$8 zOxJd_x-E+0^~LB~@8HA%!H|6dK;1VGo$GUU?uo&7^c0!v(@E@JgZ!hZ_^b`+{eBlR zU;LSDOAp$PImPl8r!j^n(&b4XYko>0cIwU4+#$>_fTO)awKQEXu%kG>h2x zhSN6q3>!urqs^2-WS;z-v)MMV))Q?E2U{45nE0QR3mN^Y-#KGso!Q&PH7o%(=GIE_#>+)WYA{p9YojE;Z;MCcrn%QK-XyntQhN~%Nw03 zI+cg_swC|DeW;OqtrxhQcs*(m*^%#kz%lz0YIhpM=tmpl*uR*&dyZu0f%}=(AeArQ z|A0|*RuLj_PfF8XeE!S`uItnc-=4)hH~9x_A&t4UUnJkop2enJg*-m%7slRHk7Em` zF>>Z&avXX>^K;nT>_PT@`T(_z9+KC8&3mKXXQRu<$?$0Iy6r)pd+1K;8Bkk~J=duMgWztr^3KltNq_wz zzVXI0=;`OE@69F07sQ}zy3yYmLysFq^KgQm6Km!&c*JPNxA=vby~4@0UC4k_@DfdR z+P(Jxy37>T&R>h6+imn}W5MPWR39>k!fhXtyktK|`o+_6)?Fy~f6j>jdO(H0v2Cfg zaxFU&o6u&)K+Mn1X5ZZd=x)nm>)hRVqk7ZsxjZ(!QB1J2;mS8)iItwPZd~VVxSc+XK~f^>!lMyD z&1_IZ7W?c0b-Eh9g+GokUJ~c2e0r7=wOwU{@haf-e}2rWOjk~zR(i#$@^q=)5w|Ne z|4h0CX$zs(@sFlE6OAe2ro3v=T2ivz{GT)~>hzr2!Ou%KlI6Luy^^Y1*A}v6^^dID zcS2PaTy9TPnu+aU%N2)Ofz!VMOS}RDHVayxxP4}eTe;m2R-|2l6cI}qoz|`VMcbyo zhc>>n!>pLQl!IvwsLFNsw@RHfoa5XEAFjJv(m?CN<*=cTY()I)eW?*=^hd7Mcqm-D zpM&3QA?b@_IBJGcBT9fQu{sPv8VW!EoqhLz!JZfP;OXC$T94L3S1jJMALYax4b^7c zPN(a&6n%MwGg>|7YBi|VHd+P3ZW(RH=fRD|^Q85WGc8P@F|oU%S128$Gw=IciWfgUaCa;vJV; zyno<0JD*(0swb9kQWsCBo7xf_)ho5DAmKSrA?u>oz1`2^KYWIJXRJlrpapHh z3)y@4DABi%W$kBU(JcFfX}@I>Z1H;;=NwDo+r1Xr-PMHvVPJml(S% zaG4F|0BuMg(8esm48r`BPC8s0h^Z1XU@{yVnJzD-MH6o!_>SB_dRkySD3y3 zXHuDxb>q#;;`o^Y!r~iH%}`l-@_!|4)8aV1mbYJA$-T2%)1aD3tl7UeWr$qUu6^I| z>dR;Npl27V>3!!*pQNtcSH$+E3rK1cMaSm#F!{Xv5ecLLW%0q~O#AX@4jnrun1zwZ zEBZ6);hTvM*Q+)qr#F4e)H%z@$gzuCes!AkzMavxcSc4ebN;k|NvURr4(m_6#rKcr znp5eYp2WiMex_H`t~9M_`P=FLKfA9rGQXiXz;yxJQJtTfL*J?57E6$o*yw6x>P=Weqp${fWrgH!+DB1ZCNU#dadNMN zGlrAYJC#k%_PuD-PJ_{xt9q1byaMc46KOcnh1n=B5>BzkTvkG&XV6>|P`FALv1hBA8~ei!(d+1=Pq-b|TDz5$s<>bDCIwl>yeaAU?w1|lQZHUpzOO2Vh;9#<9 z+3|*i6ICq)ug!OH!(mBtLohXb`;2a3ouY5DrK$C)px zupP4r`WLe7;5i2LszIa1jc6I$mV1_e#=~_iI0tq?afr4Hvgz5cH*=;8=hj6vFq@&_ zHGPRTI9R@ZIltAsi5;Wvp`Nb*U*AWWIr0=kK3hTIb63-)X&A)a$CLf9q^`q4pZ33T z{9q2sb&=>D9x}61NEM*@6Y*pB6#c0+R!-o8CMXiizoe%SU`?#gpoh9rf7D8nmK-J~ zR;&w$066Vi(!RY9gE${jWfLy3t`cLZ7at`)^Y{(QHRAj;3!23iig^wqG%Ofjq1LbZ z)eBhS$Rc~?VH{_IF@|gLbZbm40faq*=NXj{jFEmDKXXu!0M9Zbh6bJ8E!J8twk(^t zbcSK>9fMWz{o&cHDsGCLIMX~RV%=&jCX6P(YECHuSJG)ihRP9OQ*KeszhW?6CWO#c z$a+CMHZ=mNb`EhXv}Uq|d6+W%(FOzVl-v`2q~V5JFT$7$r@d z*JoAL7R|lm-os_EVD?@pHc};Zp6Hvwags@Izt8T7YZy7=W`dpP_-xu7nmR3uz2e(g1tPdLDUuG8x5h^6v2F*iY?Z!@*E)@($pV-od)_C4x%8J4Q9m z45$K2##It~OU)8M#rdloyUWz<8eX2=9g`)*fAgcsBL!ntq-Nk!jQE--Qg7)Y&aFB{?Yk2Rw|o5(Pt{vV zWnBboeEwTfkyr-}s$&`3pR4*NQV^8Mj>p#%`|b_Id)yaPYfDL!Me(bcJz9V01$n+$ zGzOfqjhtg7E!Pk?0mM1^WD zvi>w`z}$lH23&P(HwIrBjkRtoS~qKg#g!zmpq&P7dvMKwp6Ko3LNoA2hDV7u8d@l_ zyKo3_sSj_#HFZKL&dNg}2BRBt7W#D$mi&14Oi`6YmB2jDC==m71yuVjmP2RCghsvuW&I zag3Vr(pbq&W^y+56D7L= zbgF>UAJRxHxN+;;!d%o7K%WV8j_=JL?T8}Q0aD1MT^n#!x6|H8SHtvveDZ1(K@kw-_p%;uzG z(H0zBF^79Z9oOGAf>|3*sPC05Kvg9rj)s&KpYgz5H*wvd8=3I!Ci3LwQqJ-9n;-Gz z+H#6?uLEQe}L%cX|HT#`b!AvTv!?YBqY-HTE&%Q7giN=@RQg?!BTNKk~vIeP};)93Rb{!=#rNa8As{sTE%{ zYx-BDOF$%++0o^5c;JM_UR6wXZ@(Ytc_>3=RPU7xo zUS#{BLwxu7Y<8U~R)N3o_-3X*avRrNa|>hMT|~CauZHww?&9MY9uRA@H}{U2!lpF4 z3f8L1#c%OmUkLSXuMHcL*z)!oQcj)b(6`^R<@!(5b(5EwM#j!Oig%o&D6|P}ro`j< z;1||yJj}WD(`22=p=j?Bl6Iz%?ablW&QxqJ8>jYcA#Hs+`nK`JKYlfh@414oFdxU~ z@8Z~*0^F`llJ*?socKNK%t`i~IEPcJOx*dT?@Pt8GlO&Mc5_l!NKW!mvNmO5)BX|T z@N({83MUVoqBtj))NMyNC#9(radh!oiqf^1`?sL^Q&&@KKwWg%`5gXUTnJC6lXI|; zyyGbx*t3`9Me8{Ft+Wu9Q;`Yr6)b(Fk(pBh3yYfI7#f$Gn@gp z4Pj8r_YjcQUKnd22|^6MEPb5`;rTwZUnku0g7r`3AGBKl%0b+*MU9t3|U2X zG3H!~PZ#45K;BkREY2Ml_8b@5=xD+k2%Nb)jm%<~N}!9aS84q3k~Uw`JWlWxjZvpw zB>b9Cu`i`iqa+-I097)EpW^Y0IWEiQ>4IXwi@f8lQsrHS`VBt)^ivisUPMYt3e#uK zVC|Z<6-29-0#615oJcvy!Q>{m{3s2z_V>z1oD^q5CM)bhj8PY0<@$XJ$>Mp@> z-N$)u^klXdCUWOZJqRk?#sP7AF={H>5owZt<%Jg~us6OBPd#!AC%>M`Q?vKdyhkgH z8$RH}rJG1zGnZ$ViCb#h7Gg{V?A-kW)8=oc?|s9#I%qFrCce(HR4*2@zt2XqLh;&V zOnrO?5xwr@g@^mGYxb+$Fh51Th3!k-z}ob)NI6dW+8=mY&^)94FrMfa$@i~3$-`#gI;8bxBTkF)7IVw!g(Ru_;CDLR_A>WI-- zNPfWn>QcfR6+oWY#^lGwu%}@!Ucaji`+l0ovK6Z+7B6RJo#OYDRGg}`QNek9jX0E3 z)Zb5j^(rsCH;3l8KFY)0t26V>mza4-05&C$gFnt<&eCkU_vlA$net`fHd1m$e~R}D zP&$TqX;Q;Lt^nlZ#GtI-PwLWB==)wv^TAPQXK&-s4+pUy z(PCM7oXiocIQj5Ka`uK{?&-w7@EpdVSc0bQCe7)=z4-*$yN%R3nIXVY4)){8q&=|* zTe3Kmb|!Ih>K=+^xPtUEq|7@-s7o-8F~4!*k+qzBDUIl6^@-A4N=+>murY^>*SB)^ z{dB^m>-o^toY_(!E{G?{S*jsiN;#(=pg`-O@N)rx2qW&71EefJfIh1ln&pQ$^Y9wZ zy>$}peVu9kU_}T9tUz}ad`ZM*1PYB=hF}DpI!2K_M%FPp|bMoGfc#}4>QXCdFQ+{E= z&SRW7eun+}Xnsps#_FTRnDu&Mq8d^oHkh1E z{6qJ0<##&>A(yoW^QpP>3@b<9&Fznmr|_B=_;kVBbkf;yTvRHfRqgQ3j9X9Igd|Km z%@{x0LF#=QNjqaBJM|bLk)c>W*iOol9O6cGqGs1PHBr6vOmiMP#krTaVfUDb`r!~6 zvk!1ay1``&z_chs^xcE~v;AbhyPeGW1%w7gV+_#~J>W`e^bA4w%|6oJ-a_66H@bE) z=y6iG?HuM*1J*2o4e;?_`m9Q=@|A z3#se8g*KD|pDKYwjLYLF3pgybI(P*ClG+Ao?<6nu2CX0al&I}4*XF6ReMotmrxI_w zD#_F3l@ljU;B-0(4h|+GL)`8%&Ry;?i^=L@_rkfnnzRm$*FjNn7~T3j$ulFGGHS$N z&WwDEQEL>ex8BC+A^M9rf_)T39wD|tV4AAS3^C(=;FwAte3RqGnwjgQ1z+>%b= z-?C}{K5l9khegId5erS*rY*z1_*mQ^r1-Ns*Mu)+=P4=esfo*>aM#55xW9LEw4Lg3 zVBK>RW!NwtTg$n8J&%3<64%!<<1l*IF-|aOS9$9P_#XT#CIN`kM55csws(R|}mZmZdwN z;%IsdX<*o(L7G3&GyY%`>k+@r`_ zlkPIb!(MTt6LhH4gAbn_ORrFcj=p^ic`sA7nNmgW+KQrTqrpfJ#mv~gpQJW-@UWP_ zpt!Dd|LR5dt=osvR~$RRoq7GUH)$WG$GvU7dN5i9b8v3`f$gH-!v=Q1VsX*BVPl>W z41P@iX4q|B@=Y;Vt2Jljn2EGLVi#*ksynNsPnVQGc{N0J=t$%H8e?;~F`LC6l#U>7 zFZG_fp9UiXwkzTw_vQPj|MV~#XvLjaU>)UtO}U`9f#5lAvG-)|!{_B16u&z>D4jcU z^`6fDrv#YtDmCfxTbEMtjH+UFiu%os=&>cSoRv(#Y(UzO2nOT1#5(jjs?p+$yGjEL zs_xyg&z@M?|9FLe+j{+D2z`Dabzan{{gXN!5(+% zRb*$r?-Y>qmix{v-gQ^ZNKmF?cHs`I&wZa9;$dw`dmbEim)|A_HwBsyx=s3qjpMy) zS9DUU>UQF`6=HexbJjjlUe)cz>2gt2wOHJA?zPD1K zNCibSsMNiU*qZ|FxcwO$T78~?^gM8h1KQfA6|LvA3CxLq-H0#n#WkhTivdCMBm!AbdI&<#EBaDUe8wQx>5sXs({c;Sck@? zf*yqn1iX~qc99Vnq5@7ofjF`y%UxqKqc>MF7L$^CeK~{izhd)L6(sXtv2lPKZ@iJ@ zq$E;~ALGidSJJs-r^^=ci@nRW!$vddfqrPECxeuK=%rgFSlT_nz*@^0os`RvJ4O;M z-biQw|b!-c7$l!?|s;%TMDZ(eXt+$*i z%YV3PaEb%OCy=_`C0(q2=#4rW?EaMxXY8O*EGb_|BbrwW#U(>xs2fN8IPL8Hem1)b zv_yp2_$gT-v5#IouzX5&e+(4WR!AJ^MmSqtKWcExR}{zmZqYun%oyP z+wk1GGjOD*vT@5gzIgjJMr3OEV*GXDu+-o)RIJ+|Hm^K%Jw8n(CU)U9B}LB-L&8W1 z4ID~ZO{xE-9FtM2xq!Wt50lx%k?m{wG<&b0u^uI|HCM#CDHdzX9ThLuka7VCF9=ke z$FkWUvR+fH8f;!2TMd&wg1)1kWbdqHJT+n^4cm8Nz@2v!F6KyGpjD+J^?kn5bSyF{ zuyn}LNd7C2X1nwN3V;E1pI18jovyM!To-9T0t;vPzddTIpYrqXv0X?!E*QJ*ug;f( z*M1SrguN0Uak?w^&v}t5UE-8;U7|_*U+Jb7m%RwrCC6b?opRK_T;=wGIL~a(i@)b^ z6$PZc=cmxZBS4lnph^3rN>5o=Oo23ahjSn@aXt^f;p#eSx0bmg)6x2<>@bRa<|>McA> z0fUGVEf@qi3obbTJbtidHTgNVQ?zxm&u~^e*+tz`2BVCNVZkJCv8usg5$nHwpI3Kk zd2h;#^sHeZX74aII7;BYTiqn(fq?&pEp0(^a2pJMFrgEDcWWvF2Klx)05cML^4jDl zxgtz{Xuv4lm<1NeT)K$NqdE9&CVri>kB;}X2vnY{+GUjAk1Gfd82X>tz7n%HN_7NKz3U9(CS)iHYv3P$Y5EHJe*tO>%JCmV ziBY%&PU7EV`5tn`0aw?eCs<#$YV5^@wCr~uZw>8%Q34#J1*^r3`QQctkdzC0_DNd- zV|X*}f9-YJh)Gv0Cb9Pof^PX7_BG@2SFYpM13TFC-J87n)NvZjdy5uPri-fk{}+pt zUP{Sx$>E%sn5Z7iR;#rtJx*1*sM&_O<8{2yxVykzdNd{rLHhrkJ$1Rc?k@G*KmFb1 z_Pg>ke_3r;{s5@TZTBbfyi{Wenzw7$u8hW)9c)p0S~xhFlFXr_wNZ)+1?~`xJ0ymt z)wRr+Ig7b%9_Ob4;f#IaQ{HLagK<5&67|))yuS7zbM9(I^3tE!>!`tF!@6>9P%!gf zevi)YjKH{gE~6*!V%WzWXcJwXqRgfIy!{sXx_)QH>Lu*!+*$NnAvmfAk*7BB%cf)+ zcaNu6w`jgy^DVy~Z_j`l>3k#rL6d6+Fral*z$3?lOXDGV+BmW^n?bS6J-y_RNu z8?yVCAK9Vl$&1%!@Y09xQ|JD-crjuhLq2+k`rSU}u6UyWgE>r`zLX}<_2u;Pxx9L+ z4zCCRYjG~~Yoi4yu}f80joMVR7&yN8brzo$&_o}ADoU9Pv{E5dK_FOZ#n_ivLxTDB z*BPu(BKhEf>#6QYVb1Sc+1F(N5s^{Ux^)=Z!LKvEW*AT0)*IVzAM?_@-NfAc5ly7c zn7F5=pE=I{gNN}N+^Uqq5Eeu2x-|*b$$)k`!fH0cv~DUhXKmoQLDl%;>s@3Q#j0J1 zw1(nX`!lOHBoiKcnlbYaqpjOa^fs7oG4=EEfN`8q3uqxkI|Emy3w# zmC_MuJn8XZZq=FG?CmW6ZaeKCXh+J7>Fle26$5*UcApqccK=bl@xs+~+5H{gF7t^w zRs0@1Ri(mVY%-O(09311t%@jBDmiyZh(gSuRWRRHC-KP>{^hfmEsO z{h7LHD*(Lve(JMqmm^m-Gfbk+s563@+Reoq2uK zT|D&WbROPXg9foWy0?kMRL&1xh8>9PI+)j$^;TWH^TIz3ADYK1VZUpPLYDjU+P2;%9&%p@bR&8(C{6I4qI~Lm5+)2H@@nH}d7Q zP(FV1T}F;rgw~NmP}8eeG4d|zSOg$$e+Qq>wDHCV(|L5|0?e))(uxcyDW@o~x5O9~ zO~T2am^R@cA!2(ui}R@7>t@D`dVpHE@ZCogc{@^A{%GXyPaAhkR7 zdJp2>+Yj=|!`3IQo)nTsHm8YaVgDV6}b$5x~ zyQs@Vu$UWzXwMhhm#-(~^U9>xxNDV}{JaRRA3ccBka|2dc^DJlev_etW>Dmf;?@~Y z(Y%`Z4>J{1m8w!zs>+`cLi{@cl*VSWc?2lZRR&Q$HT!npN2=!nHct{j$*juqZ;u+W z)K#jsbGhB(JHPrO?euA`?ce`Dj^K;e@1}dp1i}R4`)?JEm*|EoB#hMfjVVXO=Ns5DdC z62H5|yvw*D`aqkC!-+$q#b7j`*Z3-Uhe=7B*X_XWRzw>X3|d8vlCG2+k5b`Ap{TY} zYQF{dFP8RPR=)vzWi^sN>bLlDxN)c%D2@te6)jw|8W=S$n`L5!2QYJb3Y zY%yZc_+ucI#M|;p8wjzl3>FJ%zs+LU`#Izsigb{sR!)Z!BHVuryKQ)Tq zbysTcy)M<@Uni4k8vOa6q;r9!n~YPVH<{Jsu*&~1aFXjNFF#*BD`R3}E+J9yV3T@p zuSWGk)ylBn8WUDAUvBA9B-MNxBhnJl<`HX1Cp~`jST%lIIkzp3!dy3mo7FvOu>_;{ zy0I&6|3${*4;G||jbJcZ#CI-Lx+=fYh&l9$Jszwxl=jAfw3Si((fwrrPw~67iIeGK zE_FTDsHPkq+ya;z#hUPlb>I_w(jcQfdwiwN3mUHnhYUd`V>hYkhZL2!NM}E(cK3)e z`gDG0iqg6j8)wsUP^>Xjk2GGEoW`vtJ{Eh~Ga~1@$tmO83L)IyfUb&jOnJS z6z@&)CP{N+`A+!M3!Y9&>{Z)K$qzjK^HWA9FV|s@Oo`$Z^KR5$9xudi2df&TYy2T% zd|p5Q^PknIyvF}*MZ3?EvrQY&V^lVd=MwSPE2!;C{;e#D`ZGZ?tI9uvxGOWy5^n*e z9MdkF#(>?5POok%jhY9q>>>H9HYg|%95G;c7YNy@xA;F-D%d3X4eX2tU0~q`BC(nc zsAf?3&U33*4P$jCg7p4HUVe#H2C-#-m)QCU24VDXfD5r;O~A;|EBYg|f2-pwu>lOI zol0H(OHvKxL9tWvCLCdBo#i1w_)RX>Pr2>dsOAOWnYD`H>tzLX?2+U2D#ObmwMOl0K z?bY8YswO~8t^t=OhwOIkx%v85ocw+sM|KBM=gwFK|+wMs$3t3x-`g z`E~wAPH9|N^E3q2s7sR8>`ZETpgAFWMKew<)jW&yCXAau}m)H59-dEQAhp^zKq z!J1*BR)>xxUY9_S%SQ5yZ#kOgK@(}hlHsK|%z+`K9`#3DON&b+tk)gi#8b1^h_k>( zfmnym0tc---NfMAIraZc-Ss}+Y<`wEKU_vcxD}^Y z#s-T(Icq2KmdAL!UmaZg*YnPd*&Gvq!drZn8hsz*xxu{%x|C|A;1KV>HkUmf17T(( zn*1#E)f;lhL-*1;()yo!#^-r1`ST~dx9t=W?tBz+Im*qmaLoe` zGoVEbn?ISrS1Aw^W*|2`o7hHIF>J&QRR7C3(ZiM*6PR~Oq1{!Z8Q#~$vQMY*(}7}g z+74y@9nIM_X9^1rXMzB-{qDV=UI}sPo7p?p@X_2g#NBZRcXe!r$(hHpPiAqz8qbg( ztysHaG1+zp!J*LtV3{b+%SYR!E4N?Oig3|iZ`x6oFHNG^@S8|5{jq?rF&N*C8t0f2nq4XOb_2Ukcvg6_=2R zuuv?4bI>QicCk}R-VNfSYgWA%<$XpMh<&I%PnAT;ZT+v6syI*o;Rm#GamM>+KL0ay zS7g3_P1<+4ul-RUq~yq>tHH2`?xSOk(DUt+E&*2mKJa_#@yIc#KuACuYfK!$UhkjM z@M;1{q76-NWW>k^siX6jf?2igOSb!ezox3nMd+mqC!t0o;$0TeE?Bh)x) zk)g5pX02xD*j1b=)ban=I}0$Y%I*JuPMdJ8IhZJr*~iAJG;WG%c-vQ-S2A3#KQCdt=z#FVd0wfZrXKAg+VRadb3 zgc6*PME({#F_mMe9HivhxOMD4nTb_kNbz5#;2m9w$WvKlU-O`JL?s-*oa0*IqvzG8 z_rj-48GO|RY@5FaznBU{26za}#(}S&gCA#qNdJ)^a77+QWRyS07XHRf0cf4w{~KSN zR8yh6g74;@6p&9a0R>l>F+T;R@feez8N`o?W#~8bMFzGDA;qtlPVGF(Y4pop--pPkfO}9>w$DR|%iT!j26UNTxZ1(-GaCT07KAH)~p{CE>u~z$s3S$#S#<^4cxY0=EO6Bm@ zsw6?noS(wpl?RBdP@YKPTsyLG1TT#GfxQ32dh`uev19IRh7Wv`)8;^7>_R!eY&$y( z3ME4;?}$*Y?K#f8IUn-u@>p_;yP;MTacKKDy!Os&lCovk;$wO1%Y|%BG^6(QAwZYM zvfon)2#um_Ol2x+Z!z-79?llH#z^n&hvDV{7W{tf32$N)!t)6v-pG`+U7{zjP7opr z;u0uOD5cDi68OnX;>OJzB;824<76XOsgT=DWaSzJIazrpK<4d(%_iQ~I9$TFq^z*p z?g+e!i4!MI5!cuq6@CeZQS!6fz)Jee;VhQK9rrwAcR09xG#BLOQ=vkIzuf_K`FbkR zp#f;sD)(vqzsR7Z>n%N{t3=?VR~E+v`5U*@;r!<^P>BjVzc6Z+3qj{};)XH*!Fd!8 zU6a`?Y47#=N9aj(e)u;jL%1M*U$q%2J)twTO9u(eNQGBK5GhG#Nk3OWu74EuU#^e& zk8`9<>PS6fJeR&Zg;(#^)NEOiz?QWr*C>*ZNFRLs0`Ut@;%KRIG;0}6-u65c&sRiW zMo*=>Rq@cACd1i~mctqm*f5Cfb^Ezhr3NwHsT|r|NclzGsg`n`b8TzWs$V^F79GYs zunCp4kLa*hC@t#Lq+Xd&a`wuZ@%7tODi83>DEMIP?{t2*7n;Mr@Z9s`IUtXwL8O(P zBR^(aRyn%0El>KDbL?323Lh`r$=cOF2mmOLUk;xmpg}j9`zJBz`5|P5)gi_!i+NMu zW2;9M2DPb+S79ovx_9CA4RKU%R0YSOkC?R}mU12YQqgdl4@dOoy@kh-YkkQ$v6#gN z)%5JuklV^URl!C2y!0M*X8*#_GIwn!y-LMUzg7d3xoXm7J(xbdHPHd0_DmtnV*rK= zI^Ozt7)_pP&45njnf1zh1P>lgb+3QME=W#LwYucZ|B565%pSTC%!J`XVAoAXzTA9@|4$RH;- z^A$vuD}%R~f6gHzvrtE^+T{`9l-5g4W=1~Y6(TVgrDO96!&@nnyuw8&P-IpS73S|s zCv!)bkxN>0xl)HlC1&a84ct`bbKB5s7xzHZIf+m#$;wnO5wP(u@w*0_lAup^u>Bnyd8Gtt|MkH%mrIel$T7&t|6v=O;|_wUkhH)u z)M*+dFdKhb&hsZicqZRqN1j`VSvdbof7YIG{gd^hAis!0yMYwp@37*}OnIpoqAIg! zVlU<_`jqeb)DXa-g_|Z1+Kl>)Xp4CDa{-ehpX}TMDpYR6cMH6^xUPt5&8tvee|JR( zN0dgBdx~pCGRk;Ae5D&rd1N?J$vG5HMcsAgkNk%8USp`_%wff%H`%Uh%*n52P}LJo zd5@yej4AY(@GKo#bYqfgB8%HEq``M<7~8D|4nqNQ4^I?%iTrl*5;Hzo$B-B=8dRvk z)!=%FP$u)_S|%Kh=kuc{=vFoma{DsBe|0*3vJ3NPO{m(mGG{HVneokZ^y%&C-*gg3 z^WNl{Vl9*L%0bRin;&cY@Q-@pLW+bo7_IsDd}=WjP-r!gjTpigY*7tp6=bBq@XxP9 zxiy1qzX;x&Je;4t{+P)LEr`)tNiU8?bs=^Oap&`yGkqvszWb7S+b_|jVl!lUS>y#& z=lMwyy#Lcol&7mOWN0Z4oy>w>rBN2ju%|9y$PgRZN|^4GP35?4|sST!`{5 z#w>SDPh|IL&RvS3yJie7SjpYr|I9U=>>=;Cfz!W7l9lF6?)okLJi->nO@EDENK6eQ zSGuESyXjv`;(iv5bnh|3mC2>l5xOR=sa?y56YG8_Lu)}Hh8Hi=5j1oV-S!Wo^4vxQ zJ}c+em`xlwlUSKw?<*1H9!|!$b2F+1 z=}~CCsnjS2Ph5@&qLH~6?K!GRHOf7q(q1MxBFOHI#4MC!H##ZRaX5V|s#*2rM6wJr z0YGU8ELV?4l{GFVB_=o**|7WRIw;4mn&t}K*Ol&qMsb`N+P`(eH$`*&7@GZxHA^qV} zzLeqvRcoqNZ$pq?iHC0_Wjw7?K+W5fpK=Kwc^M)+oe$IqN$cZ2{wo?_t3;=hJB;5b z04xhurSl%4(PT1EWPUQq(JAtBtAoI%gXrI~I=_DQ21x?u@REZ0T1mWklB35D@_enD zymKp`b2DQ}mgd+=T9K3BwtXm*5YMh&Pf^=9kKBTb#f>wd()4BCC!aB4Pz%DG=@b+Q z!a<2`@HsyIiwq`B|w_2#9hE$lZ$(D+dV8~fn0Gr0Omh3iu;3Py6LZG*|?%c z%kKgx=|@at`|pBJ#YubrcT<&#V{Yfh>67eF$do)2-Q=c!1zAP=74d4o^#Z!AB-q{v^ah{%7)I%xTNr-V!ehmdbuM}p}AP&^NU%B6_d_^ zvqm({$M>ep;@xch^t7a$jP&uJ%5@r z`(s%&Wh~DI7aI-R1k|QeazE!&=Ou8>zZ<`wy2Pc67dbMg4y)JC;eQ)Ay_yx#8H&ixH5Y>*r?k#xcTssULUMVPI~{kOi5!x*dGY#oA6Nf9 zEOkv3!xs*ioyWOd-e~hq@u#+kCHjr@n6q}AGaASmlijALTc`RB&)|=Q@_JT*997NxU=h7^PmD%(C`nQ1%|om+?pFH?$Y| zy`RBx>Ni$gi{abVZ{YJN0GAub$alsPyW=>TL2q#MnJ$dz(^g8r?NEi$YWQFl4}YI# z!%p*Th@JDh1W=$U!`2__a%Ia(7JsoA@^EAz&EfnNcMjSPRg5^V>J zq6vVPNqqG;M9J2uKvMEefqBH6!Fcn}C z)?Z4xLIAD;inSGi-b)I@pw=j{n2InMwF2)IgaI+cjNFw+@`)mhm(()dl^`59<8C%72iXtaJlyvM?NokJ!OfHW$F(}jQ;3tyu?G#)D;u-n9aY96+JkiWf>Pv}YWlo25KYhDx_ex2wK0mI`+x6aVulTnFh7_^~ z{$H{^_x90~zLP$qtI8K;aX5qrkHI7Q?It(fvs`sV`-kIbTZ+nEgte4yB-cyt%HUoh zhK_S82ms40#YYP6@CQojOxgk+Ur8b?Z z>{LkcmZT^n!y=$t1-y(`(GBWLjed41Z_^F8)qP!B z8I@LtZ`r5#aB*deQ^W}e%miFSOqn^zQXjOeW>7P;diBq3LV~Iw!%iX$%03S zKf20VrZ$Qy(I9vM+q>{H7tt*PRH>{!B_e+8=asI4I37(=`Rngm)_Mi{g~5c09s#}#Qz;~Z+gs}$pDoA=KX@0`C>E%4=;iB z2= z0O6MNQ{5cZ9&%h_QmN7p(!#=SG1z1K6T)P6AU4H`ndvRFjhDcPjl}J%nE%r*GKE9B)62tou1=(MpRCABVeR5C*?K-5pK2X=`{f=) zdJ5prkjjoVi&(Wk3C~i^81h;#Y6R*qUf#*#(@tW1Z?RVH9!>fWMUCv8!Csc;3 zPCM5&FJb<#2Pvr7hR#Ae^Ofc4*Qqkc7S1KNVNaf}9)`-4&JQbo!y40!{>{pwuotj@ z#X{z9KS6l?_6&PzfMl0&e#25O>dR1Y;2`Rz-Kb$b#3k<*^lTi1QjDu~>=))P{h2GK zGW2_67|+xUktDubyVvpcx@{y|gL&rVVf3kAx;UtvoBme>r6&oJ=_WVbrkktZru@h8 zoI3h5mDT_E@-$bM6Vu=uhX3{x!y1JAt^2M_y~?*AearQMBbin&@)5d(|E#&hJqF{q zfB!Ryg5A08{sak(cKNbpe79&J+qP{Jq^wUkxc?weYD6SzW3n^T1c{)KoUClJ({sod zL`8j6Flnb&Flg-&(r=t$>ZGMyS40ygOJ>T%$xJ<^qGkOu#Lu0=rwg`A^_~BGJX7Yc zrE=3YgykKl{jd+%e9eL?{Q#RKw@qc_ho6xvNM%cA&0y@BThwpf0H2G(#-H&e$4zoPbFT2qybt(l zX%a0P#c+MyG)6DpOo~-jtTA%^zOiAU(C?4QC{vAwT;RRQAF=D?1*uQR4({T3a;7Ug zwkehP9m_cxpCR>i|Foe@|Mdv?0eeHUQ0@maG=F5VcLvOS(XkJO6fVdX(w}z zrbu-q?_0}jgJ+W#(tsurNqjx!bv9;L$lkS%x87QesZw)VhvqPU+B?iUTkt@Qn48?> zCO3&xs_P_T-|57StbYw(wUp|@rW2bOT07)#-}ge9hH4%9@>bo*$7mJ*v-TqQ*gQQh zWWggf&hFfHe~M0@IfKz?#M>-gV{~Q963$FCIk9cq6Wg|JO>Ae9oSb+jwryJz+xEn^ z{qpX;e|zuMUDf?nuUcJIU#+I)=aZqJY+Fs2_QwfQ^0t?sYTjcnKF~?*GpQ*4l{i5# z!TwP9r9iZi7=Vt$s#6hHiYpw$T86VYg^Ruz-8yTznBmrrGVuFaMZhygi;ezxKP1+( z@f^;aVVx^m5^lV$8e3`5Pqz4XcwH49p+%adBo0!e(0$XbT;C_`;KiLtS*I_TS!Wq- zI$`uJQ!K$Cx~yoR?&a1hvt$9FF~UK~RRWWK7oG1NA%DxT8goggZluUlgBNYKiNLza zkS3d@{Z6*{E%zd6aLJE8L!3T~^tlC~il7v?tJ2p9>F>Z!m`W8f<;h{OvUs7qt{?_! z@!dYXRX9XRyjYt8_<{MC92fC|$&huOeAC_zMQaor9-6dlD_XBqQzOk(g-4VR#vy0#=?|942fux*V zf!E#fT(Q!QR1fCDnYq{s(7v$+yv2Q>;ZMt)V(3q z<$NK=LYi9gg)gnodOkGVX?+s3=pEfLnu%&gz74C2XTYB%*Aq( z1Q0#a{VQLnmVCq03x29T*aqx6-vadoejM-6?g}lzEhLE5Cu6l?H|D) z7sV0qwZe|Ar3Lpv;IMxNwpv}uIW{Pq(uy)LSqVv^NU)t8O}n*awLhfv^F9t2haA!y8@ zyo)Zhh#lea{LF0Zer`BDTIFOttxj=>9$dJW=ZHp`$8P3)|H+5<$<>YxYZz06 zN7>&T@oPoD4Q$ga)XJspELHB6&PN85k>O!fusD!%P*YYQzB(-+VoxaTi%MqHRH0rnZ!V=*5Dc)YNU}KNat5om$zv%jw+JxrM^Etwtmq8UQxsXgJ^eV4IG{3ESMJBMcB<`;B*)NR0d0s_67!lU`;vEftGH($$u7vGbfA~2Dq7vY<*{`zD z*g2kK&N3N~X5(FH7Ht7DIwN>-6$q=4j&nC9EJHY$Du_^w?xY)ch8^Pe#I{(Si0`b; zKu=O>h_7>r>Sm6@;Qj`nTB5&Olz+s}}F!cX5dkurpf zPRjXyLySRYeo3xm_jk7tT9H5LK9ht)2FRcdTxSf#Dd?%*Cr2?b6{BasQE?oh%LxBd zZRz6KBsqi%)&{Cx4F5yh&b!rbXEe}GXSZh>s@0xf{G1!nAcqvvPD&DbIVDxLtn85i z+s0sWdq2n{8pTpghyLoCZ=p|K^oN>U&UFNcBHP0->xoQ25dDB^DVh#hLUu3=69(rO zNQ)o@HmpF--!ybr-oj}DBno|kmiTKnLCjBSz=kPihao=#X3+Ch_#8&%cb0e52b;{b+h8zhyc7T9een>`#>76kbLeIRRB)6n z2R8Gf%^t|UvdSLMHf>xz|=V(Y<4KQY|t7= zV}np)zgQ~ruq%oVJpSy4+xHmqGvA_D#`PD|TY(>4nPSK{*Ke+Tc*;zTzpoptDl_`M z!TBkEOqgc+?vNJxJbpLPKbMJT0lC=1Q_R_u9HomHp8YHwY8 zuv{j1IK@qpcBr9tY2OV1I@xoxPX#f$yxcJlp!-1ylfSCE#FGuq@ZL#_`iAR=TyVaA zn0vynS0$P_ZOg$TsKbv6;>=e;CAR{NcydlOnfUrevno#1nyDRq3)8C^#e3h!g!Sa0 zqxU*)R8sh>F`Y?yhlXG*bO$jlmpgGq-28_)b2be;R$At>X;;0}Vpa|*I1#IoFZC^|~)K}8Qtxps!f+`T#ywfn_v<3)kDRlU~^T0Mvjc7+6>NHcLq|tuv=Xn zQks21dRyp)Y>jdR1CN`G$XZnd$hCOUSuzE0H*qOkEgW7CPH$P1)T%#ck5uE~WE|33 z1ZY%$VqvSb=c2CL+e=`%gJ>Szf1}pbgtiOk$(#^Fjl`dy)E`3B$eKe<3KCM=#m=TH zsK5XYcKjP#h_l z&N6~B;UEo3tak#y4+Li4ps4m+xEWYFOJCn_@RxDN)!2Yxd4<9J>pp=$xMfQ5l^s{H z*h9?21xxD2`09CIm`y`BW<@-zAXt@YAj0?}cE@yi=iK~0yijy@GqjjYh;Y6@?OEi< zQAlTh;a8581nEYB)ak97!fRQEMv$lSwLt~}Onx|!g*`syIE?MO>X*r|*UY6ZkJf6H zhcAV?%E8P`%e$6u;Fk~Af~1jE4Md>GTUk1mhPex{j$?UANRBtefLuy=H3v`BG9H8M zp3j`&%Q%&KBhr(djwAkDdj4qD?XE{)TD16_H+^!tFc)N6qBB`GPY^dmOs4~Dmu}3o z(HbiR&?9@cdDpmYS@L~yR5MO1%Fnm#FB7_PvVZW~2$Ocy!;WHE7B$v<(7E1db}d#5 zlOfE@3fo?R#op`O4N-wRLSfGu{HH^nW4m~r?rTQuNjrF{`=>YNH}?b6mzcu2&Z%Sj3i!D$t3M*$i; zf_{~hn%CzwjM@#nktybeX&Qo9mj$alrWdH$RfU3U3DP4~4#3Sjx{Ig#XNKp+{^&|z zfXlW8|Jx({_sm&TBm1IONZ`QaGH50|d)vLW3WaW1OZ#&+T&dl*Xu~o(6j>GR;G6N5m!qrb-FQT&K z+8F=>qv(E5M|gJ>lRGa-I@xW$Q!Cfa?Y^))*FlFH1?)zxA0PHV-{Zu0KI*$4&urTM ziMB8rFj(Gh1bc2^YBX`y$ky50Q)qSFd|wc@ ztK1sIxXXbDY_EYc}{oHxJU-RQ3y;{P?jPK&d7}!w9lI5;H@99ej*4?+w6+x}9A3ob>?-Gd5y0tY#RpaUtSWmY(TJ7hbTo6G z*{+PeCflj-|BZXlsgWYT?e>T%spSBu!Dj|IFaR!VV48obv3VmCMDK!ggKc9yZ|EShxSZSEo>uY(x~wAx z>pwSvvq3wDlJ{r#aB8kXdT*079r3^BTDFDy01KPk`iz@+y>DeuTF&+^q?BVZ@mamG z;5{^7h6n5#NsHFVr-V{v(DbAt?rV9v&h;qHT>oZ9X0Q46wE0f?&vUa#fzIr$ z33%9M=Kf(Vp0XX<_M5!BSEn*G^Y3~Uda$@W4kmL86U(zdJf(o9<-FY=BzA$@QARBZ z!VC12LhwC~LkYzbX0hJ7A=4UR+$5>jJ*-ECBkk(d|5=``N@2OVQ<1X6b(_4EW{p9 zNqO-u;8vnJDkjfc?v1+98Ltz=zo0^teQs8c1h=64>hpmv+1t`Gj%-_Jqmtc6&ewqLJnY!ci(yiH;1WR$g4f(aLqq{gnd)s`h&*)pB z`LR)_mXGT16BiGPw?2Cb##!1{goaf29Ob)BkJbY+^M9@H-G&LcM%6>FCpyDW}23N2CP}OnHmCm|uz@RB!hQ; zV`(xtF>U=eYdO4}gcxe806by}xCWg?jz1J_Ew!H2y3%ODRvGoTInsuyLGVwKD_tg9 zOLg5?`eo@#5anL`-7cUfu2q_x2Hs9u$xu#O0+0MD?UX08?blal1TA=%_YEpnKNrF` za`08P1Z`)1th&I~pIEb%aa1dC9`WWQ2L`j%IkfySGrxoKq}8kiR+D1~D&)0@COcUI zBG(5cIsLI^)W7n?Dkyc?lD29(S1;`aQ>nTSj2K*yk~g!^jUq>~1%|YbLfzBxPIG~9?_vy8JUKOH(VQaY=X*s%(4v_(I zGX21oV2uhM`Kl-p$R|s>T(3#jV!DeSlj)Tp+JUCL?P9j{jILq`d{Da#CviJsVBosZtfwB|M)HspCEgFkW zt&wN@$+iICxO6lu&H=BOId_T1n^O5woARCi0Yo zZn%llCRU|V3zCfn^P(n3#7&6*UYPZ{xU?Q?HXyeKcgc_Qi&_hXs25KQmnnGC83y3y zVi_0owM>2+_*@w|*BA=p8lEl)^zKiJXgrx?LAQx(EOVqAPqP|f|A(^^ctmC>4#@^6KYUIHU^C(JgsC8b#w+I^hp7{ChH`s=P2 zE70bBqdYhL#%rH0W_Q;p63JmV+0R9urUX2^ws4doc?*a52kNzEhwtfRaC7{JlO9kV zmPW=J- zhCVwl*)5(i@d0Qn;(2ph`rt#Jh9s6}6L~UdfQhqmwX7m{Y*_cHZE$je6IY#mM#3*8 z_U%{D)lLu7txF2r0YT(%AqB)>TrkR!`PBuAWLad-4~1PM%jV96F;5F#p5T8bP-pc~ zb6m3RI$0lanf8R)x1>kC=dB#hb+c*BrLAAQ4Jlz9qPQCz*VVNW-ttYC(P1!oa%H+330F@Q%dc1)E{JM5 zygxUT*>iw$wUilU=2&7fB-Qv1pVK|*J4D!!&KiZ<9G@nmbYopluo;sX4cmJ- zp00mSQZWX-{33@E0)Y=g*v!^VgO;dO$6>gcPR4vXKefVmSn4m@qrILV$Y8s5v<=Qa zC%ksv#%i5okjT%1Hc!hOTIkSHr3lTH!7Z6Jg5bDua`f{Lz|kh>>s;-*#n^K!jz`nq z>>FhKu{Ve){2P{N=wmPCRZ$9VR57BzOkjImKhNrQ6#5YIPdV)!iUGX`h-sEO)4<{9 zDQ(uQm59YcT?(lM>JV;I;+M0uSwT(G)JWXd@BUmx$MQKp>OpC0@AEHrMjLKT-E$d) z%!MkBD1QgZp)lAL7Ew{xH{AST9AczWdt(0zL&yVQ(7s71y3trcv3KY2_m?QW-fVP;Pdqk<+meN)Dpc@(qB{e z4J{QGg`6r1ShupZJtG@uF6&Uo3p1~)X9>=QGvuK<7?W&qYPJ!(Pv*1X?)Y01xm z5J$oS?8nfAEXK=_kiezB1l6(4cAwYtqqZ?CN)FN>66T^p5CWmSeUfeJxTxDt71I8gIlGCYb+B_k;xCk!QKe)BFZmp%Q<~d%ArI^R*7DtN z7L~#j@kQ4?GUfw@7>`wY(HpDa3CYPmFLXHWPa*m>pyUls z`)qi=D7U<;w*-Z~l{HW}r2yldL63Zkzue?(kn^i$h#^()g)0fyMAH>eg%N#659z=7 z)ET2q6R-@Uo*GEyO89NRL*8n70H0(qW+d&Cc<%}N;4*r0bcyWjrr$oKk|ULc29C~d zvuYFU48|->A?Ho1mixwc(B$X82B9ZtM~G8<69vS=nAzMgvRcewVy(*VtanbwCS~8i ztn6}jCL8TUGdbNEYlm2V#4j92Lk+sbT?`@`UWqBBHsFud5A#ijFNXTh&Qsj$IJ!Yr z{aTrj8BLsGSkb{c=&EcWc~sE5hk3JCH=V9eqOL-PtQY8V>#v*BRwX)x{mFd%-Znv0 z!0=m{dvO!_!HBL!p7ND6!|61Nk`pbfXa+3aSOHUYM`r!dn4Y)l}L?$=!YqtCV1LU$7Wz1m|_^56<~E_;8Y! zPcDT+6TNyX8dw@oYro2)Re|wv$S;mVpVB<^l>BW+`JDY{x2fL7X!0b=(=q7cP)Uh8 z1(I8tk=6awh795LXVr7GskjWQ5Kd8k3W|`3)DJ%%iX@2NVwXhvH^X|$@l=LkvNw%$ zTCwiy@4Bx;tFszksjpu;+eo=!_QJ=QY~N5ihi!(utL8-*e9Y0wHT!*M%)<_EbDL5P zfQ3FBeBzw(rpkH`N%W9^=%$&jaO8cju$&VO3UBh0im7gOAjBTJG<=|7EutNok9jY^ zXqL|$ks3^A60{w^0a11+2x4i3RyrzNTK#2X1b-7f(VBfQAute-iTL_rrtR=~xX(*EJ1y*4Et$slSzTVrtNCL+4gdh zZy?5n4?T}GrUa6@h$tmXy>jbvT=&=&SNNA#PWnn`n#?ia)?Ob2NS(2i0HR)RW8LF5 zGbLCWs{A3KEwCO57B%`^67W665y)QVPsYp`JhDPs9Di&KLgEVPQz_SK2*X8Cz;Qd5 zd%;0Q!+Qa?L}1iXUwkWI>+tkal*pZsq1Ebp|eoIggsg4?zD zZ&7!=7qAza0coX^XI_TI_G*NUGJ3wuZ(FxE250iNz`hP^H~Q3~s>JFJMVkA!+3T5mtifX;`q`pA2t7hO79KR!VWBNtyIFjo@37_B<4<}18Z zBK)u(JRRHy(gCjw)uQNhSc&HJ!rkTWpc1q8zF2#WIPDg45pQ=QX~wRDRO$4(+~Bx! zy@nKAc?BHTbCEWjt*^<{yJNrFYWhWEy6ov!opKMC+AzmQpe5ynq*Z8YWRj{!v1IA> z@POvfBQJQ;eYLRL(KXiN7|IFnyz>=^6~=+c_<3miviGmef%KTpq2=@8pM9VD3_KqI zW?c-hZSpVSY~Ti87K!3{JVewOYa5Vj zU8>R@&DQ%qSE@dXsaXr}f;ProVlBFG*&c{;%eJZ)!(&B1Nf`a40y zAjWb$<};#_itYz@+`U3Xk4D2nWW*z^`AV+8dp{SNOib;JL1^ zH26GTfyOd;k{$}>_WEjee#zy~UlU4*!ntNv4u$Uwwiw02#J2pr`D9ns?jY{yQO&p0 z8wNQy#nH=FkerdoY@QAuv@vFX*R`04gc;-Y$oPCMXLcTkQWmXt@?_0oNFGlbWDS0X z|FO{?NiMNnP5b+lR`dl~?3fut&iChoag~y4{2z-^_njPN5=>NZ1I^xe$6hV(*#N#X zw73vi-sFp^aWPpA zAqp0i+Yh?frS)}mPe1T>ifo1KCQSwp?3vAvFlNeGj24G|F5sS`8OgaBh)-fS@Sn`rF3f3)qOEI z|2MfrJkDS9Mx;hJ2QI$u{FEvHskP4FuwH1^?Yyey0tYASJy5*CD=u~}pX8Mg$wbQ2 zDNSJ{VfTtDc;lRkEEY?Q*{WUWa@KO=5B{A&Ig;->^7luC*j$DPpenR)n|pOuV-6bz zm=VLtJt6eoEvdF`tO8A=-|-B_f?@1y$$P{1o`p8BnPLbZ$Y3wh9$kAx`L*3~?4!au zZU(Mury5sp_P)xy$ya1emNEKG(C5Xq$3--+C#4_)5f^qD+x!0G#=dHVqbq6mf$ zY#L24HpW+3a$Lpa1*CO zgw|y{QCv z-cF48a<8gsmdT()q=js4!=?XbiZZEsW#k{MIJtq^_i!Xxoh7U793Bu3ce;|9p}{z9 zyL7^auoz4J_m8^BM!xH!dVInriTz&(Ki;hF;#!t)z#i{ikBlttGJ=`74aKcb(l2+4 zn>iK0EkXg48$-4!W`JajtnV+m*s)LMoI0eb7uQ0UBf%&^9(0KZvz2q1sUvE-M%mof zTZ{c<;`pok&YglnS`82~Bmd}6GQo}A+uVd6vM0ezF7EEnIIewqQykBH za)!U9Z;ujB{jTZ1s2*{KE%G#xjQNPACIuZZuoUabq-%3c<8K)GbQM0XFZEPoS~xyF z6ImZs-W&nHa{E7mlj1(cHL8qX9t~D+Oj{1?n(2rXKdpJUualRo< z9OH4nKts3r4=FIt?}%qyXr=#_ivc?|rNQJ}*wr;2eO8Od;g9|W;)^(5VBYZeqe*is zP}TrjvO=zSGQQ=hk8r3ZH)YEr;1(DEXr!VNx%o2dS*ZekJKToN`U6S=#eS+V6;WM* zvGlWr!9_zANx*m3dl`^q*>XfoP+G3*uG7{WFf~9uW=fFuT>>%sv|Jb1swxPok=MNa*z{m0-j3HN|>g zB1KGU9$jt2rgL6@7IZXG_M*u4N~`HM3ih|SvqnV5S7-vc3*HoA;3`#9kkWZrflrA! zv<6Ay!!_m4$E4T5PCV{(Mlj?C%bY-A+TId}T-24_-<~RUM!vQFPYEgBb<7*w^63_w z1P?myqxosfNz!vhxy3HF#|g0Fl`h*yC%@DMp_>e1&{unk*;vB3Y~rPv#|ZrPtY@gK zaB5eLtb&*kl7Le_VCZLiV-lX%A}7? z3gGBs_DaY=(iAnQT+8P2a=Rw+b?|GVjJ0qI#HeJ#luJa+ zM~Kpyrb-{jaHtF?=(w3}mH?j0%GGE4ZS_3b%fDhISB_UPf zKUZdP4Ga?EQ5D%PH6S3S>%}=~a{jPvy+3{v9Ch?xRy&?$^2Obpd`b>Hg}cvWnJxhp zhu!Ci+?1v-vju(8NSkgEffa98jBr4l)>))(8BTMG8&q z+0PSFbTViByx=;fkwr|m50B%`{qe?XJUsYHw0^?;j`oEu?-bXJ)>B^Ww=VxRrpkpmU% z*B!Ix=!}ukf-w&Ye9|&bqV#@C2?`CFf@0X6WUeQ4zgqtO^q9{Eh@Wo=TF``NXB4QL z&eA!|qMUnF^1(g=%XD9R>tPM$B5Uaal37m*XI+KxOa^5Q4f!BB7!^phd^EtsuZ4L^ zjSZmM#G|wi1kKS`Y0C|qDa2cB-mG1`ROJWxhNJjY5m4M2ji!p(dHMW!!L?P%Z?7f& zns;oe;!WK9={pHgTX}E$f=7)b8E8KXdnSxE9FlGta- zff-by1@yZa%fn9|q_6Uaeg+fp0GuVpfKNU~O?JzO%>2;><3Y+)*iT8@ncYyedj3x$m4L+cpj8F|rmrr95&+19_L@Zu`t#V{(e81Skd(r1gMrWO zhU8nGImBeZhKI>lsO@U=bMq~6M7 zgWfYNwdliNXB{m??OEr++baojT)n59A$yka|ZL6{RpK}e+PL&yTnYl2qg`zHzxT&2J+=Ia69Yq(zmV*qEqUU0qfffVUs9H@Huc9V zWKk0>e*Wd_qD(>&k2;I5d6H`H9rmI3cV@)XNPlARHsAea#ba3|Z1kEYGwH>b`y#qj zT9a|2EJ^d@hCPQLcJ=;4$fxir`ul`hV;E}~@Kvb-@ohy_%tVfye-Jim1iXd)r;>g2 z-p$&#kjV5tZ{L;bmrFAg=Zcv&5u!m%_6tEL z=?xd@9MGV>W|;k4mfyIH3B5FFjbLO+UMz~t<|-llbdL*_{RYuGorsOBX#veCZ3;@CXkfmV!{L+5?On&?Ua(14j7QgOI)b%_ex>xKz0V15ZjkpilDIzC^8l z&z-m}fWxki5X|7vk|(`i6Fq7JRA|R`dSFy_*zfhiFwRukk@7zp;=LsTQHerWnXp>o z6KQ#|Trly%bI<^Qh*=TMAvdb?Y6;*nMF7VQ->`du&&~V^3Qlbxwmo8hy92gu>Yb>7 z6}7MThrs;=IDho^Y44;y<8g;(BHe?&Lw|r=T{luIcut(5aKM!CHBcUocur9m9%+^* zIDokj3w!|bVWZ7-`B-o9qBA97f`2%cuElV9?557gUB}$r+`|3+WrFA82iYP4`AA_j z;|-uc@rcC0@7khkgwH=k$Z*_v0kJK_U0!C;zJd8lqxb2B)!pf74z65{+;j=0#Hv|b zjXgJK@y9KW58gq?8fJG{QJ++D)G)T#w4Zzz$43RYs|`se4ma0DA&VR~Jaj zKYdIN(?GhGD=Vv(;b2_KWcq7barxa}m`xdBy0%w>(t5!l4}`yEr3(kGr5t*TN_zkl zH|Sj(aenid+VC(qa&^GuemxI}GpT z0qydoBcgDpu(U5s+liI$%6EAzfxveKfx&)Ef1$v6`5Z}}P8VF)#5hhHOFZ|rYaM|1 zV4b4`J!)l3Lify121Ua5>5IdWg2nZrZJn&Gha_L%dctI*kcNYIm8LbOYIyr6S!J7$ z?mJRJ1N&Tkp_aU<# zYR^#^h?4od`N#WZPe%HS8YZ2Y4z|aDTKeL+`X*wTB2}~`!h>boI`Tj-yjT8RUct8= zl;wo&hnHL8A)^n3DZ@^Co< z&49fv66RK+PN_Z*sR@a6Z*NZ+OY?B0h;W%jS9mI*TW{c4t&G|=O^l!oX{Tb~lUu>M zI{9w(L~?kXWZ^K+@JNMW&L%Yh=K4f4}lh&b6MSJF*VwPubO+nA2u0M)u{evF)@Qa2M?i{HLP`eGW*%Et28@Dh8wRm+iTQ1(RKr70ZQq4ANaGugAmx=KKY?^Lcg#wj8BzxiZ z6&`o;Ze;@l@Jzr$#ReH!Mi9TqbGYsLsb*WME00aJ=-_mPJ)x2Uyns%U9%oOp0x#np z7&1`U@i)ZSG)d+z_YrtO-|S?57S306aq%q#Xa_iGdbr7=MsHt{sU8Q1?sXwJ_<`r_ z;ABGTsJ(E*oupU(oJe z6$uR;FPs65T!Opj@uU+X1R905Kg`1uhw9U!!Z&_b@f~dVObuRM^WALDLvOCB$isK_ zc9Y^xcvh8I^$gljR9$PImq6b=E&H~cj~G^6+mW9&LJY$ZIxFB83%Jd;w(&68h%hZK zR32p-rbbOxwQxT>PtsHMtH{~X+x<$*CqsYJ%Hf;=#^tZCyHx2G?7iJ{i{loGgz+N!ei)8=Mrh9NzH0LqsTX;}unFi% z?|MqQpJ)3E%xsjU3)BDDbn%?sVSifh;c9Z@!woxA#GSP~BIY|^P8UQHT z%0DzkHY_Ejy14N7=21IiR5^ze1*oDUI?@RtZ?k$v8`XGDXw?w@TB*Rc`S$~wU5@D=ZtBob6wJKDfQVHi+BA#7=!{M9n zoau*K8X))Mnz0${SS7>;@9h*7n9RX#UmE;#M~Yba9rYZ#`@7HOn?}+SR{Qc5I~#-2Y5~)`hn+XS+zy^Dgg1^Y!1M z@E|!5Ky8X32h9yj<6mta$&EtMhasH9 zDeDqy*Oo{pFV~BP74O{qk2vAIvY={SAf)o_WMM81mSu;Ig6qz|b8xOOi@J5Qqsz$Q z52e-dipeA3AtaWx-u)*sO>}73Pf{yuJplVR2J{Q5f0di;-a2n&#jk#jODqaG=bg9x zcc-=$N$C|pDuVcE&Pw_7C3%zf7AVGySa2n&|9&_A=vP@)sq*5+aQ( zA``9aD29{!#xFOk3jgnpPcz8QD4R}IFw_6ax3Rw*iP zqi2**pseTNd+WoQXgI3D%_3||x)_Gt+MKK6kV2lr7@g@XSCK#obDWMmOp@_bqbD%P zesj|JM@R!sBc+;`kLn@I$4+m#zK7}>s?+bN@}$b4^Zw#?s2f%aI8ZnjT^4bCqYvp4 zs}ni4p2Z)wD@l?oNtUssr(Y%hiZyAE<&*c9tW^C2E@a9zgsOQHMukhsgR9GnM$L3# zi5P?Uf6qEvhRVfb-PSFlsF3tHRXZ!S*K!4Qk^0kQ!yoeh+23u;{;70LYdI$NGBCAx z$NzB1c2&=)ebNf6lkxRub}jdR@ObW5vFV}u@pI*EIowrBJF_Pa{9m9c;x%lSG%&R8 z2dg+(VUd6xU*-V=4KdS%73;I8E z<5?p#^){C@B)rrLvV+3!{@Yx|nH>CqRcp_1JV(qG#O{KBc0_nuom;gyFo>BwqV~W0 zhcn)VYE7k#n&~L!*4rsK;`^o0EpJo&W@M; zGvVVO7)+1&Jybw~d%Isr4PFI2429n9-Z#TQij+eAtKv7SVS$pCJ?K{3S+g^#s)jwe zMRpX0nHjW31~ET-w~py*4MkZlDF=xrZYdQ5F)>JjU(i*HW!4S<0128Zw5N+Si9>I} zMNj60Q~=RM&-q{`(fNlRZV6$}CSs{Zlbz?a`;So-SJwuHRAUokX3G(OnST%u{$19R z7|ybyylD9@CqztcvBAQ#SgLl*)mD!{AJf_l%wYD;7DHiND>pykNHQ(9ptMvPQ(jpJ zO4ynk`=0|wfLO;t4Fr?ZHDY_J5^@wRMk`-abO8VdXutc3x;c_GDX;* z*XaFvUV`s2L|Hoa6G1Rl_H}zDT+?%SpYO;g%d_z23*6=p8Dpbc_cIz)(aF%pAfNsU z|3@0Ma03S7vxDU#a@mTo$bEhtbVUilrVQbnr6$W*aA97*Ng-U->4l`~#`Lsd)0vMs zncu=!uzvEu6~>Tzi>&WqMdCT%Qh86cy&u^zD?ed~=Q;GUcIKv{tR1*`z-PH#(%6Vm zH0wFlr(!Ieo2joT2VJ0ucO~gx679-=hNdtMQxbGHP^?JB&F|NhwH<3bhQX@W=Ei5K z=RTUmSY*H12~A#+!Q-y1sw-N^g<7{%&9FS8q0CN)P7C(R@U%Yox1Le9|UFm%WbJAf5 ziaGS;2b)o&W_2hgbT6_`PVa-K1ZLXx>(?)P1|^CvKh)+1UbyCJGC6FgonBOhjB@5; z4Xts|RJ1dq3ppH91nL+N#-1&7g7X;A;E31Q6XH+T#2kf?)f(&qja!Bt*rce4Wl!x2 zH~7L}CtA|(_v6f$#PhOwrRJhJHEH1MOc=EToZ!Ihum7BJ_DfImK zO%w)lh1wxfhd|>76HSx)z0D>P%(p8=nwp2mBqNv3s)~+XVP00M>g+(_ zX=2-NwzHZqMxHB#CiC5F4rg_cHuNLSyOI$%bLKYTLq$rj2^y-3YFc58e+SL$V$$Oy z=Z(7$9-5YtQ%j78p;eM!=gP{jG($O_i@&>W$F`ksAr`@rZ=m$;z&iD%<(lr`-zqD@ zXDWXAsThQV`Jr|;R#ZPAs4Y)}aizW`8AiH(ZZe;Ec3SLs!bk>Udl9F-Z*5q6?^BUA z&=Z}={$hHIBU`(_IC+!ZscnUkUXxLLH*R#cBK(y>3lYI}8F%WH0pNbxVe|=eUo!A%$u63HiHeZ)VFltRI26zbuvA>f=ax(bCkqsHVJWO#J7?}2 z=F8V+b8~Z3B`TEiTu*3~&D#Lw*tjdUlnpUdGqb`Fd%}*42>o=Ss%g~bt#~Chh6?Dp1#`(A`0D5iv;M9eH=Mc$3ui z3KR;0Dxs|Z^ayiFO}bd+k|&Pn%%*D&u*$+wGo2tzLzPuobL2Kox|W3BiMAC5K0{9-XeVX?StgL!$O} z{W6#zbya_8_Kb~_YZ{{-rN_yyZxcT0k8UlJyv>&Zs+W9JQF{M;c?+XAn3%IA7lrwW zTTMAr<}VQYLl`Vn+Y$r*qA=k`(*PYUs`^(kf6!49N`G`gou-b7#p9*Wk_-on@0%Ol z7vhETt6S`3k?EPIQxxF=dl~bgzx;h|v|4r6%yrV%)ntj6%SlubUIrr=67Nb8URO~$1`s@?U>6-YjDY?=yZpM!s(ulW|{`z zUe(Jr%3>!yi^}o-leXJe3W=oNsn?iazdCgWhcE*Qf3PIT%Mf;M8wSUvTci1z(wt6{ zLNqz!_Egsk0b9^5UM;;MEqf3o_l;7)miVSn!R#tgnxzfi?9?$RjJXAf6Lu@YC~Z?E?nrBI8r?gKI{beNh@FE@uu*~-a`tDm-NB66v_Hbk)EjegOBg-pJGG6e+%L`O&OcKDWg zvREukojMhV&CdM~JkX>u@v`ELS6JGAAdmd+N+FyA)YW+EtvX4)_|0E$Wd6c4neyEd z&hY!0e8&?MC3WS(dtPEtr#QSj7VzLbkCPvY<>b3x=iiwz7|ni`{ref7d+%*NTe^vW zXbfZf4d?RzyNl~i8%^2L|8n!+J|!tVos`sAN*8>~{G^*1`Ct}*yyZz!;{7a0JemJK z(T}0)KjV&ne1W~9gwWu#c>LjikmU$a6EtE9Rx>_``-QvP% zHnDBGIHwQSlaP=|MsgHeXMV}bbDm_)u)YLlzrk$}|A!yuuAtBj8H0xNt5a^{fyrYj zo&E*){rMR-Wp>6k<2i0!F`XM9dx_r5-sQ%BP9-|4D_zpeEcs?GiDPeN+^+_+<-J#U z;+2>9VEG0DLg`q04Ca!XZ{v^WO(4xs%bIup%Du17Va3+1)c72j9f=GdeHITrd@Eh; z;{Gt&iLd*PXa0UGI~T2HM^O>>tUjFcm)p7J!ZXM)c!d}ah0T;gkOx`(<`X>pV;+u# zG_sS8%>H}^8MnX4vEA)FegEH?`NINsmek=k_(%ff{V+a;c?3m3o**9q}dg|)H@&*fKQed$F+MT6udtk1rH zXVkH(bx|meYAe!_7jERBd1f39zE669K$n|uX3QWzKc9d6;~#pXBO3@gEIKDAhi9I7 zhEq;Cg&8xyW#fho5f1a3E}nMosA6PhZ^rbCA>i^h^wos$c>*|6vKVmu0Cp5q5bAX% z&)#)0wRQDAI9kw)q4()LbNBDCiL|_;I722gzJ(w2+^bLWx@A1Ce?5mqGe2QsPd5+W z{5%WR7L#+vH9YpGGug6w3EPNa(!W1t&hwYk@0e4WeB(G4uBzkW6Hj13b`-WDXE46B zn6ixi{Pu>M8JZB%y9NpQ@i;R$@keF@>+7m=cRZ2mD9SUK}k7ApNG1bT&{WGX>K3WA9Gy}PrvvI|NM17Jb^v0 z3$r{KT>8K(%$qTlXD%Jh#{d0;cb0Az!om{KwG|@Z_0#|I`*>_(Px97pr!v%oyFUDm znSVN)gv!PI>97A_xi6karp{&6vKidj$Hmw0Pv+HyqFlyY$aAmX!^jL58zUzh}{`uekA82hZI0 z9N#a<7dJ{VFF*02DB~2K_-HP3zx$kjos-DcAKzq2si*#oMP1N}S7GaZ3eSHujalD( zz*Pf0Jpc5kEM8lI*%I+W5enjRTRG*nN4aT;ja}Qek#8N%i(k#+mCH^izGO8Y&sjUw;SF zGEU}+ujaAj$7%dAw}`*J^scy;rI@V;?x-(?le{pe9@a(_D1H})~G4<;e3dK=xdFoi_(?;>s*3bj3jks$%>h6dX zPom|PJ{()OZ6!B1ml4B;bIVOPbK!*->c8*0>n=_^?KJxL@2{7|rY)QGF21lO!Qk1& z;vySPQ!)wG5Rt)|gF;*y>~Yvrx{DuP%&}?o>fRZf*k5>^p?y!c}amEXN$znLau3lsN}-)&;k4*{P!$G$0LA zTm}K7CY6OLwtHh!J zMN7XU&zMGzV-xc?FK3r69?yTi1T~VfyU`;d7K;qYAOu0cDPz8X;Hh&!teF+yBq$%uZ};4u002F)`=q--np7m>66tY7LXLJwzO{aOo_&w z-J7vD+`>6yMliBl3bpe;;m7hQI-PJiV|phN6W^K9Cyl}pQ^j{%DzU^T(OVol)-38} zVrK^Q?@Nl)N>q~go*9qL-<4Z#{v8v>jG|w+ROZiKz_i5`TzbW+4DXUobXFIJ4;)Eq z#Rj%j6yd=y1g)K*xLzia$JCT;PM>rix8L&sJz^Vt%lx&JrCh+IOD<#h;O_JoaV#TG zIsqy?LOgq$^p}b2Ey^!0rw85R9HJbv`TgC0X2P+@&?7YtEp0IWdFENZm~;whDM=*s z7|z8*d(cO#z~>EOiixMkfW9QzED)bbzy1SBHrKIy{T$vYPU6O~;~0^fL2_Cb1`F{_ zK^aBc%H_>^K;3pS`uIZ+J;bhEyC^Oe^{Tv_|NZZOy3_cbciySTU}~2T>(vXMdjY!l z(7W!08iT58=+g)O`6M23uPGFYBh{pIq*}uEy^X29VGp(_mgKs9`8r9fQ>1u0lBa8t z-E(qO-Nua@dF-E$>w@XlTW;ky7yX7Vxm|ew{r7qI-FNjsf^wRONJ~P{n#QQCI#&9M zD0OM1M9$tUtr9gAxC=IjQXeMDQ#8gtzqm*mp@#C34$oe%Kcc>cM6wzdV=O;B^=IZh zDH>ck7quZkbeBwhID9bV)6>CX!)~`rd4k&)(4B;xox5|w`KR*Ii!<2ty9+UVHI2%6 zC-LWwC&BNbCgPGx%a`At#PFN}41akV>+;UU^~pkd9)CUO_gTwT{}p0w;&MJJ66yCl zg`O4<(qG;l^fj&vgUle?W*vjKrG_LWuY<+5);0w!@al;{V*wuV0Za_p;s#@s`sGhe`al>T8P)IHTxiu+xbP_ z4dPll?BX~cH@<*|$r_JU+rSr3-pP+EcVSO7Q?h0~m8M+uC4Wz^;IYRaqoSgMgoFfo_2{WL z#9^VMX%-UF;#stG4YQY*({Dr+H26GOtwzeW7a&phUnkZZS#HP|k+S5>$a+H)= z2)8`}3PXDLh0o*Q;+y`D0r7Sr8UnJA33zPec1|GV^Xa}s(i1r3j*wkKocWz;j2L=4 zr@!(bAAI;C3Deh;9NU{~hi3@ksBNW0Sj@aiJCF!CD8C=tArBknT zA!CbuOi)T2myS*V@A)sWs_5ZMl|K*HszShbk zY>5grjNUX_ajkfl%O*WcAafY!9yglqIjNNMF7Ge&(NeH(Ibtv9t)?Q>nd`2)o)fx9 zQBx;GoF=Z7AxhNYXi;wd{dyUc7N#eke3JR|=j-?Q)~#E0;WTdCIPSgoUQ$z2_s`lZ zM6LPtSK;+3;(}RZpCJ11!?)kW+BpmFZ!RTdv8boMLZN84b!dP0Ibh3Eb7E;<$WR6H z;d&%?T;t5?m z^}G0Y^ndFfo_=)-$Mm|Dp|SPWDRaqso`3jB-mg7|`;Q+^r>GD$;&`s2GF(eG5juVp zQKJ3bwPFcAIj&vz^{|naU?SF8%#P?D{AGN<-Oa6RxB{YSHW`J$u{)6F2xDxNen~Yv zdIRb5Xygtb&cM^V^W;-c5|x%h-}A1dpWVk^+DTRzZxBcCL3GO=!1M3@kJR2Hx%QZB zqRa#7JKD%&Z+=9#^f8>)KVBCIa;QBN{1^Ot{th9Y`r@?P zMIETY;gsPABPK+j(9$#$L=bwpnPEMQr zFa91I-##`?)XVDrwRTbndoxu~&zQ}u@Zs6oU+`}E0orE*s#vk;s`IWbs zzcr6CR~^e{yw7tlf5@&X58g^QIX(I@YG@KO=gj8)sXvle;>B3Bf)8GJg!^9K#^9lY zi7uJP^KX94>diYvOJ7S&r))BkVlin!sRl8|rDEB(mQ~AF614Z^hP(ejKd07wpvX`V z&6$L0%}f@q+eYbtt9anoL$F0hQN3g}%NMO={rErez!|+o`UY%5C@lZ#f4n;NJ9fHC z@d+UnmzGO`O;w?&3(+-v{@TlYvvNBnbsCn~PINZz;tR2C+LDbF z)dtXP8T9I$(9pxD>47GT=1=F1|9--nJdutoK$rA%(h{R_CTF27euvMdeb0uT&^zpuZ>mn$|>RTm;TA&VhhVkuZSgO}d>jCB>o1PvPY#I9r| zM3J8EV9C10yz$XY))#s(iTyu)`El;~_cC&N5205|5f9yXFP|=6MX8WeRU76p_1o`Q zwbRGME3PDA{kJ^z!Ix~w&lf^5L|XS;v~@oS5&AjHH*Uon^pTL=l?)*;X$TQ&wfI^4 z&4;`rj=8qdLs`I1ipW>zBqzbmKk&gkaqhczux0H+X1w+k&wjgB2yQQ->|u=Qn}MZr z3t!LthBeEIp?nL)&VG#Pl_<{bb6);joSP7$THb0te&InTPhX3n&v6V(6hgLz_aJ%S z`-F&*^*lK_xzRd#FMR*~_x$#^zon?Ch>VO3-D$k(b}b>h`O?zT_I`VlMX_js(6sh? zAE(nMVCpMg62waj-Ma}fbs2%!*uD0X)?fL8=9HpTO$@A199CMK&PmgsRuV3NLFao@Do3mhb}3G~gFsVjoOHZ4*ywufah#fIX3JFp*))&Jvv-^^q2!nsV}R7>yE&gJ4Uy{Vr4H21x=m{@Cw{8e)Z_8P(H0X=Yp z`~+nelF3RJLmsn!`jPLuU(GWY4Z+l6$4y8Nq?u|cX8M9P^uG9yOzathPdb773z)Zb z0moeR2hQp$8cfZEGqj7BUU-4emT$*q_ON_Q5%%F{aza)NS#bfrTe5)pv*u8xd5AG< zy!Pbl6j@^^-@K7^&XJrJw}prAd4VmKSloFVnBnTiq;Y*PhU&$I(Q0PxYm@oc%&k~7 zFM0Df>W~asd~PA4c_@={)?1wZlix?+;X zp3N1f_9xcvq*I)iwX42n*4$+*nLnGkWu3VE&wt~_k*O?v|52X!dIO0LA&6HmA^XG$ zI2Zhzdp}r0lrzBg74wK4K7nC5DI$LXz5J{;H=jKA6z?zJA%uXJxvRzTdL7T$-f=i` zx?;;;$JB3rV9`%PT;vYrk^%AL=WS-0cL=%tmvKY5b^LIK z3>%KN=trha|C$9tm@M8>LYGr6;?40ZlcafzPDK0jyv9osz z1tcwLKEF?Q4wrG841Nv>n3Z`Y;c@Hj|CrujH$L z-NeXFHhh5q?&?~qYU{8>#}H+==p&b8dtfk_#qo@~mB;U`qfTt+4Vloa7NVo<(&LDr z&yBo-(n=E&=Om+)TlKt!#4-zZyNMN3?&gNSy~2oRW^muhUByoR*7Gi8mkd~GFk2hu zfs<7%(6G!RmW7pnNO!~zHUwUjO_QBKU#xm|Wg$`yL%!XO~@c()nMMgFfR*1qnO-&#U3an=<*@&7`N>a%e*H8(TD^D! zp}Mjfzqr!2m}nfPATB*YBUTxSTu#vlWx?$90K5D6~s@2mqSfX$^ zEP5AaFw}aTmrmiuCB^!SLL-qA2VGle)ncE3ZY8rdx|#HIG&XbYjeoXKe)VEYI#5R^CGM$LQbg%f?$n=CC=?Vo zzLBVeIHGpvQM|y&?pIMS;u36)(hVMffkxf~^`kiz3WcJr4JmD!(qMlgj2}`l*w)c@ z`!nTATS6EdEh6fG_T8;Cd{lca44?E{MhAWR9A&|<5{}%ip;2x7d4!tTUQYcHB7fok zxc2Hhs2+L(_dIqD{Zpd#7*qS3@3t;$Hvjvve{+JPJ?lfw7vz4Y*XsE;x8J^wwXgg) zvwx^f*B{lDrBEn3fC@Rh0%x!1;=!I6IPCw^9wD?Z;bc4ttx*SFk3#tB>KcZvAIgZT za?;xn@jXZKzAg9rOF!`Ya7lj^)y^nddl-d6`UKb(vqkqQ zlG73GZ{W*ADV*~1@^r^`IbpZ5uqhOZLn*lSTRb)AFs?X41XRF7U9BnyMLXWiG)*5x zCxc?HTenWPKxx{6KXIhkkxt_Wigbo|L^(-MOV_PrN<1kPiZ(yreh$l1YZj+pg0GW8 z@r!&jlhe5fQIeFDL}Fs%L40a)(B`=ci#a8n6pD`iK%q?@D*NLs-XVORniE2;S)5v& z3omjQwn6zkDHKQaTbg!|{qi(brV54PAnCHh{5;VWPHO=p!|M!63@H=}g+ig|2rqYs z(D2+AVYXk2C{jo%QKe8Q6bgmnmn6*gseJb3+oXJ-6bgkxp-}vs>~E-ogT+3BWbVs3 zygA2m>HE}?!xczhj)U&cp-?Ck3dJvxR_3s4Iu%R(=JgT94ZA1#-7|HHA#&nh4$ZVI z;pL$g=Cx~!Ff;8-Ahp%9bRF{s3oLZMJ74i7%BPmhwX8IIgukiCUd^P(yIy*Y8y(z=#D_ui*ofXa1J_Q>zS z2$2=k>gUcQV!hly&>+6*r@h}Jn6&Vu@iY#L(loF83{}unG&Q$AQb2WdWFcgxXDUZ` zg+ifF95$+}tBH<|Rt0e|#oN!w?ob6y#l*hE#gRPXE)WQ)TBlGb6pF)!G&-oZ=*Z=e zsUGE!txzZw3Wefm)EEamZF*BbsBmAQ(WHEe6bgkxq4=dE=eBIl%GR6!YU{G*mLJHz zZ4qUW>r4l$aMXNL3WY+UP$x>rE}+6%A@Z-eWvwmW*Shwk9l3sA-jOvj&ifC0l?JH1b zYy6U?Kau6qv}~mRm;95<<)@qyMhdL(GKE5+P$(2fp?GSqU{b2O^|CP-O_&`vj6N^H zkcQ^>;q`|wm`s=q8j&Tfg@Oe90U7f_tdnA_J`8=(=M(!jY$Mhg_nJ#kh&G>}K!fwX z!EC`~7TW{^^=XLxePWf#WX7m9h`jI!0JA;G+NX=+A5inM<>4yc>dM#Mf zM8%3uCnwHEO<5tOLM)kO8!nVhP|nrkbdVGsi@&6RGO^BR3ZZ#SL?tE@ZIFV;fUB~Y zav>=6lJ?fm?-LM0Av!6MXw8UOh>6;gZ4@jn!k-a`Bie{1CX-}qfWqXz}Ln9d4dmD=rcC5%Frc*Lji?LBXtS&Ca zXR;F?ZQh+PDaeeao0wmd#IaqSRC|;)P@zzCOzKvD%d78w#-i;tn5|CYhn&pB^CvJQ zvHd+f!OHEt^Z0EP^!qh$zW4y$4q;Q>XhPk9AGgOV+BdCHbZ*i5U=lCKR`E8!YvU$r zVlwC%6C^)>JCz|DDV?%Nu$w#FHEEZ4YRwWQyXnGRR7ZuSC{)gp`Ab+l;~kci3}ftp zh$m4gLZ$hv`}0b)qVDslDbLvd>Cp#~WX;=EFM7eZ)vw*9A=IRL7QEKV@f(33I?CL_`2@x({<^A9C*1>YQ>Dd-pXUK;xN`b6MB3B zFpGVpc}=P!QO8UoCAZgyUs{SxBCV<#W<37|`B(jpdj~{O}4LPOQVmj@Kt~OQ*Z}@~po2T{YDB#qBD@XP{yof4^Ze*T4A^rzL9C z22`jGg+kH6tWC?O^WaTav1;J8Tz<}2OrBC+d*?as3|V;Z_v49aXL^nvr*p@3KXBW8 zt9Z%&`SMY-b0-THEX3n*h_<8t1>ImU>NEPL^&QB7tQ7qvxhN%;9`R0sH6?7>UWGp) zRTob|cMZiAE}|2Xa9Fe>Kf?IYXK@N!CSn+M-Xum3PUWJ>OF1yz8^pD0B_(w>#yoQc z8CDNFeD5%?qLz@c8$Hh*O7-G}*hh|{^Ef+|#UAoMoG|t^7k|x*AFpCX73B8n%9WQ)ptq%rX;VJr>tHYbblC|6zkZj8=9e*S z^l@By)?f<1oW{FzHekz0rE>jNR%wY`b;;?R&^MhxNgnG~tY*=sQt}rrVx71y#i0ax zMNFPE#S(}q6wbxD6d_b zOI3{zW2a*{Gx5*dar^%m`Sx$HSJt6A@DvI~$Dy1L-~Q4#w+L>XREhZt(eY}&pyvjMd=JW z3JSOL?z0cDdHy1d125;c8_p%eQlEbHmf1Z0=*ujxfMYJcg`3VCfQ-^qy=fu;c~qoV z?I!)0v0U++OBj%D=!4*)8V&78Jb#_nzJ^w zWQA&N8>8V!cNjMl6B~uq;XLq8tWbR!xeaa#^GfiS8%VqELe99fuc#JA%w3a-jx$kJ z=cC}0*(~1VC+D(Jbn2CaEh`O2t_^q3IO6-H6K6LOG#N#D8rGCloZW3`)*33lTEmK$ z^6@+}k{-t;QvUrStg)HoUeS-3D3Om|J?Z)KFmn4x)viOIk{9{?Mw8@e;Z#uuZ=SSRWJ-Pkj0mQ9c z#$WFI4@-S^de{Q&T9e22svwCXPknv)eDUK7%ET?1l4xN0nk~Hi%~}!%^A%`7oFoIUx`T)RdNpaG9o%`(<9r_I&2?A)iuiSJal+#tu+^gx*#0#W zuX>0rQKxhLB?H<1-hFg^d3JsKY&PLqIEz&kPEJ1gWS0N$Uz{~%aed-zr*ZdR|H{0S z;hZzRH}5@lKev5eK*+y}e@woc=}E)6_=3~$eElC@dhJW`BsEaBt(Y~%?pE_+wAo0E zjm2cJ(Bp+ z_$bn4m)MENGWJ+6(Scfa_=6a$UHFHj(e<%0*o3h1?(I}WNHb$`ui-zlDj9h3rQAP0 zmw*r(1Dr`h494OSd0O<*Pjo(U9QXcp9Hwdy1N(R3!5iP^*#)(HFm?!8r5mVl2T2-q zEEB|=)AwJBvh@bB4<1PWl*Oc4j|R+n4BGF&G*=%7Y@$j_%nw|LYt~DS9L0dnU0Jv) zs6R2!di+JsYJ)D2gLw?%MYyDJCpgS{yW)&VBheB<%gCUpWCN9TzTH-;1 zqN3DCbYe1zRturJovhlj6MJGRX$~XRB}Hs4bkHL?nz$$z#Z`4!91iT!QCK1dR)s{H zT3u0r$7CngZpH6);Sw*~W+5m;J}CW}#2027u2F+53srTj7yFF2hcPj+E@K>^->&SdpH-?Q_JOUShb2vkHd@X@nK z9<_;On;q<0G?$8)!3-G?qmPygS>?z&3(jaaYhIqgrq~_~yKgi>A&y)Rp38YXe5~A1 ziSv`Ctn8LXtl2{K+FfiF;_4W)5tq2sn%yUEnh-|2cmr@oVb)}tnsivhY_*e?U}^AM zYvyvw3R6=@f%tw}PP(`O#Bp6FaxR|02i^b`6{Qpv21(3~Ax_*d6++xtW7FtoH!@!e zP+y2TpN2u4iyU>R36W+rI;2aJxPgR#6SrVyN{kTqLERi`lle6n^hzq@Vwp7qDWXCL zh2SySEjTnkl{G$*D=*uFb@WQkptIF1Zq9}qPdd!|%GgvL$6c>a<0R8s9(}inijan_ z-WaIA*~D04$Q0%BPOXpEHJf@^D-?>3gLq!~LS=ZgKodvFVX~6a*}_Nvyo-|f zG|WqWV!PHCtLA0Hf?^EWf7FGOB=t{!?|%l`{Z_t25L zv&PYH=yCM#6h%TtSJKvRBd=)@$cMGjNlI1%8_Ft3Oz%XL%|O|PVuA((R@t*+4p1ka z{}t7>kRC^Df|DZgH7>Oi35|LbLil`cTt0)?CPwHt(E!hHWU6pwb<87a907l#&jbiH<6+fUuES*V;OhCPQaO;>7Dtp$ldF_st)8 zC)kr`CiY>#*jS#Nvy|`GcjbgkE9<_V$_ItjoS7|e->n$7Z)AIZKJMrcv!>_cH*}#+ z+?>WxH3fbj_L2%wX_N7ZJm!_wQ6b7Cq*+Dz8S%NR$S*D>C`8Vt8B6%2Jf3SV7%YU4 zh0KthwN);Pgh(~)w)P2e<+gL_-=9Qt*WxCJCm#z@UREar*zS#}@zhXTVCHx7ElADH zs8A?65dP>cWM{*e8E>c99(l+DwA)!1NYh2GwIY3#6(pwGp|~_ zM4E}oabhgnEVdr5@8jLRiUs*_?6JM{7jdZ@%ni>%v&n?67{v25R=oHYkGza~@AwUA zMh|9-U4NO?+!gc~ei6?edpb+zea9ysOySmzyO{FNYe{Hx0C6EgLh%e9F@j`A{mZa3 zCY&oD^3~Vaf_4b`!$KyYi!i+d!5^e*$5v{^%e4&K?a`HEAKk^=hreLyr%SPUg+S_@MDBS`U1`@YT*2~3W>NZs zsDUe{vGmw1(*AV;gAB6UuK7yu3&rrO>&_tW=?}U3_DysSl~YhAu$@Jl_$JxLGilNI0v`3WRwxu5hJY=N z%decoM}K{h(f^F&lSwBMTfB<5o_&DX{cm8BKbmcEaSR-FGJ|Z}d3WUu^4!C5`8E2T zb}~zDdWZYwj^xhnbv*vJzwmPJ8`&^ChoYilYHFe=tJUai^H5MwLSWZ+N;#gBQ~jV`|a|W@!yR)cPE`GPMTxX)UNHu-nJlY0)qStfcqqL995A zOj|P;bqi8J2qQzNaY%%)F~eXqV-nALX}Hi65BjKc`!NeO8SuF9Xf~qkCb54Ik8XLQ zK{SPSf0vdg&ExU-X*j*Mh^NTkO(T#;E|X4G;mH6y65F?LCtX}Pv$=I|eJ$i>dtL!G z?hsC!30IAq=%iFKQsVa7wz#-hj~*Ty8@qd%C|BL-Oc#{mBK-9I*J5EJ12a-6uJq%` zNF=Iz0@i>Re?b|(I+Gq4GvsS1(-I z0W||bDY(UpgNab*SRCTwiyDG=^Db)gUHENL*t*3L6>GyF-YkOEwRrMt#5QJ({s4ZF zrq(q92ccHoSPePZtGS zODPkz)fyi~PD-qPqXo@&Qamd-=ah2^T>d=&zUCCnJ~#U|BFQ%g7hm3SHn*+0n7K2q zC%U>u4gC=>^qIPpA;G%!?FR_arB?Rz#3AEooBa_2p-kZ&~-tn!i6V=TAbe**)O zJpA{;Kk)cBB_!u|B0JH~##yTv_24_)IVPJg{_`mJzxE?3(H`8rPvg$JZeh^&&p7*! zPf#wNx#K23#-!@G+&|?LR+PqY;>FkU_${Zg;=R9f`&)~3mldzk$@zEvn`;MG^ZLEN z=Rfm3#8_vVsKZ)rhZshIDhcpsT>(;HKbLY<3#mlwkt7Y|^SuEaC zP4DARWN_D{M%&hv?O^$W<&;O~FnVAPTG2+9tlx&U%OHmI=tQ78kHrh~h|bI+D?5W2 zPXWs|6k$(HCo4Icnv!DN&J;SwTd6GD#m)*J@u?Z4#@K0Ofb!N+URI9R97T-NLcm>z z2Mf_o8zFD4Xceq_cb<$(6$pi6P>HMHtM_@57Ov2t6i@O~7TA$0p4vl8;;BZ6n{aHb zrOTEPlMqL*o;~miVX13sf5SLFX)ZN16>ri^8Ekf)zFzJtZ$zWk{JPzqc7}R0rc~v! zlf1zhT>ov-_C8*l%yI@jj#Ky$kujI#y6`D!4eLUU*2#AXDbVV>Go_D{i~}Vrq(Pjt z9@|M&P(7-;D7Qew`AYvRLuk)2!|9r$V%ag}G6sE`#LQ>zp--HlnLwINErMd3{2#dV z*Y|Vg>(jWVmx)@XoD>R0hZ9c?&n!~@Q8WvQ#I7R!Tr zSc7P>Oh1|s6g72q1Uw-O;srn2R^N{LMSJHrHpJ_TK+fr!ot>>~fvO$(ESNW!TB{Y4 zHJ#&59!kt^Ra4E%MawBFcZoL3L2`674nrjcb(%gKM(4r9$kgiCx^@kPt^m$XJ?WJh zho^8G+ls37F1nDAF>&dgNj255T`ZF)91;R4EhP!3HL{FD1YEUL)Vc^t3zcYZ9F8dL zRueuUq`b2IY-p?W+=zTx8XU;|!BG7&X%*X-cxp{BDe=^@c=Gvtx(9z)JZ;&sMYqId zW@f61fe$(cY?fM<&Rj+EzyTyhTleX|qzliowanTQ!}#Hu)cS)MR6!~fibI5a{vU{V zIueN#PX`UXnc7UQ0#qmzii64_@q^w=>}WzbX$L8-`t(9=NN%6v zsjZU(t4Vz;td3-YE*Djrg*e;MW?HS_$G2Az=--{}iY=^8 z?!km!@kdq`{B=~y@CGsQL^;}P77nrLhefQkB{IZR!P3$s&O9{>^$MU+C=^GBz&Y&V z>6ajq_PXfaeBQnJZr)tC0}ocR2TWx0nhIY%i%J zZuqhEi8m3dEoIH3mFzO7GHOIm?5&=wi;q6uOj1k=zS())d_rb~cnVRweKXU)UeB)o zXYV}Vqpa^g{=R*=yX-y5WQA?m2qKD#TDPT&T4$}*T3c&f)Vk^(*w)t8-M>~*6crh= zmjVGofRKFfM|d3*Xw!R-7|mBxchzX`}>YgEwPb$g8C&e zepD(E+P;4o{FWwGFWJtHIuDVtAp}R67&vwaSs{J%HnP?1;kHTN5&OF_{Pn)<6TK#nc8WK*vN5-nM(h6CKOUQn zvSK-rr=04nle$7PJ9f2W_j<55m2=0f%en9y9}$@$w7gc$HTReBVS}yfi{Nq4P}WLr zZVSZ~)^_vj(NY}8n!p!!(HY}b4o$*zj?G)s< zkeg?wzO#28Cl%@0N(Lo_V>AX~^dH}!1E;Nm2XDEINpsfX8j_9b{g?Uq^|QEbS64nM zpT|zk_B=Lh*hqfS-u=&$&p~ZT1yzk*>p2_B*}Ns6JuOy5jGsUt_%RX{6%}xdZ)$4V zS1+#jBnMR;ZE`xD2TXDl`;jJ>%XPpc2W|D9P`-3K7dJOhXY!zRET=Ovh9P&I zOI5+wWc>1GGKXvMU;YK}{A@S7o*7Td<7=rLJc3!TUyPh#%6;WvR*Zj#ogbgYxxXJl z*netp?Ji`yZv?uQJ(v~YjJxJ6LXIh3wTzmQfiA&BYQIdL*L7K)RL&aiez1_xn_V?S-o8$^j9(MnG1uuQwPK{lU%kSofhtB5u z$?<3;vDUsikNaNO!SZ@F6JnvQ&c%#3uHf#m27;q?Y665q&s=(+6^Y-0?(TdRYDTbG%Z} zk1e_Hk*$-yL|2@H_9r({5)~W}jB8(hj^|(hjJ6a5?Ry7v*I%FHm**x)&rk&)KJfzI z<~1?y&gs;C`8gl2-$2@L|H7ijZzo(BtpozWj|`no7l^iNX=$Ocv9arqq@1osp*Z6E zBhBl}I!F3^@ayr;>y7$gYf)c^=j0v#sLETYwfink?%!VKv*I}Z{6b2fcjv@SP~Bc^ zcfHGJOUf|6cq5n2`3bT$(`}Xx=jx#H=>>G27e?E*a#RyD$)ufzwlGJv8t6k zaCXvu8`abS+&jiakVYPL3}M8CRL;6Ei~-JOem1k1rFp~Xe{KZ9+1dQ@0X^5v+Qi&x z*?joMx!m6(ow-KIl6ot@Ikz7@DMoz}30(h*4!2Gpuy!RTC35LqF`U&uh!0-g#tZ+h zW`-o55^d$&e@_A4S!eO>J<+(9)pGasd6ZZ@-4-hkDqRfcJu;j}3|0K%$`W#anZs0V z3-3I-hBuX&tXVRGXl%TF$ue$yY#SRsnnbkpyqboM<{Id_y3MfBa1*c@d-n@?UhLk85+fs7w==vlb=iC=^nh=e%$)!Kj51-g2x_g z;;DbW%(qX}u$%s9gy~5j5d27y)A-2gfldYA=dfL$El&pvt%JqWkHt}}%cRGcGn_b|g}iV-kwMXc8{knX(YCm$*EG@a$vT|*QKS{0N1URC zYQF-l!ryCrRj%8k2o&0tQ^e~%q4XaTM&7+k7&q!_VzLXVYOqOiB)V@mS5BhpRO0gN zzn6ZkiOBAw#qrU#RJVIjhS;fcJ8?F)@!&0+q|@@#WcE<&va;Rg zVOnq>@_fm4l{)2e!ux{<WW*5xhxc8kONJglBg$gnD~yK_=|^Oy(b^L%im`|8gE<(kqS4T zZEa-6pww>bij#J`7rjx#j=vZ2jwAxM%t|9p-^$7r8xP1Z?l!wgON=B%zyHDZX*DFo z==sa$dLFnbiKwnrwRAeHxP!uoEG^^pMNK^U$W*2b)sW-f!qN&qodHXm4x`^fOQ(W_ zZfj4%AOm`>eDyr&Gzt{^GJ{LvMp82dX_><3$7@wFbw(C>NT}m6Y$O#p*q<|#gyd6!f{#yb0tLzlySJ2>e<7zHr`MNs3UDLpdHM>~9rUY{!)qN;4XAUEK ziso5ze>?qzxIcD{M@ zX^f+$6TQt#aD<9Nw}r-zb}H6>&cB|RLu+k0E9=c*-OJ_`U$bOYJ?lT0&SlAJ3aVPf zucJU9I4QI`6#O$^!G!ss|EDy)0R!j3tDB+TcJ%((-_gqU)r*<;(R{vMo=f>&JD$D@ z9(OxsTi3IA(E?Vk+QgnF3w@2%x32hx4_N$HT>UAjv^K5!IY zfu&m!mi$(If6wzicV_OLIcMh1xqPB@KZL}e$UE`%#(L-tCFQ<_@t0D07e>_Ywz(7) z14zCd+O}>HGkJ882?~i$I%&7hiT$%o7g~UF^M%iFuQE@l9tVUqMRr z*BuN<|F&TQal|&b-{XrWTZ8&H-4d`U_ws{AMi(1sFD}Z2=}(ZYQofva%2*H6zYmPw zEezTH{JpmOq)Y1y5r>N<9nyO3*k!&`p78od(fXmuY;W@!wL3^1KA)$MK{n4QdINVX?smJN&qFAyax5!oVV=D)GKigS1WGWPtTH4oKkk-+@ z((@_S;^U1K-}#tIm+8va=SN;Tj`|6r62S>;2KD;y+%o((%rd`W>lQ@LP4r~A>{}jM z2~-d~B$dq61lR6}I92-$YZ4>sNH;iUUn+W<8J1LVJq!8pIa6)~M*42AtqC~9yOmvR zVSS4L=nXtLabD?pe$+N{dXQvx_5I6trkn=v%S77 zAlT5kMK_3k&&%ga$@!}CI$-rE8=-9ES<4$5-~M`5r2eW^hKVfmQ2Lakpx!(~w^M)R z!uTcdrg!K326y>x@)4QP1cK zAll1C3^%MP+JjD5cW#Zte+rA81S~^SdHJQ_#bmsjDvlbVrxmRP?KDQ4X~_?J>;~lq z8WhC7Oo;um)IH=edwNo{g{8#fi9+c*?g0daG7=w9{62#aM5I-qTPo4hJvKzPG8%?@ z(C++()D@A)vX6Z=#qA*_2zjOuNOXeej6lIPZ`@NdJS=IccFx35-6lDb+(T*&m`L?ib10mCn03}4PP|oj zY>}4}(R+BOsCQsI(t4j%;P%?4!4<}T{PU%>s=ia87!T_DWEt0J_M@$yEAG+oP3qG? zpV&Eh4MfE?B4uWV;&o(?#WkMnbB7Bx4w|7bc`s^u6J)MD5 z+U4Gcyhf~e6AFycy>G6I#LfvArg+PjX2R?H2=(m1&X6DS!f(}ZWx6p6y;4LzF)y>c zS)tLm#$nSJzJ63&>4Ea$I$(tQ@cKG@q7m|>AD1nbBAnFrEqKV!!3aza^`^kXx*xPU zbt+4B6}u-j*D~L24R33GW=`<}$7{CraQQY$ZJTN(B){!m)@;vq$%DIeWvz5Iu)%!- zQ*N%8w4|*kTDP~Oz7mLCRCfpHqxr+(U#0<_C~a_X4QMs2qqTP!$a}^J&c>x5VGonv`|E{Y|TKAdw@p{LG5|a)>w&Qbak)uFu;4 z@+EMI1v8EJJJY_VZ(7)rMZ_z+UrPvr8D8ir#YTuz53s0BPcJ;@cJ;j}X_+?y+Z_TrgtiHWb>aw5ykSUInt*2tn<Ze3VQdbZiVXeCcnRTY#&`nK=^Bt@7ePIbT zUW-_N?ziMR8+W(g-r<+*tUon9jQPRf409D&)Ft;pV5U0 zJ4gu>!qPSiIp4#JU5>OUJ2;I?3inSgh8u z#i;5h+7f3*aB$Bjs4Fe>kHvUc#c>1tOD3wlxtrgP=p~vP(^3qC-Nvdh+0}tIR4ftU zisF;jSuFwmYOSQ8HHrguEhu)qf=VpA_Xn>BQ@J^5xP< zogo;&CW!TA?1L$V+FTVgV#~Q`5UGr;sX7?CJI2&q46?o2*yyo1l8WV2K5c1M)c4jw z0J;`K``F>h*-B=WNQx5kltY)$8t$;n809sI$o z;J#%eB~y9km#t`FR2*l3=cE?RW2DvOHK}oPUTD5%PL1oNm)=YZ&AUTD`pTWa7hbcz zLmhz!A2SuLN)mqz)7Od{o-RKklo{#jX?%-w1}?S6 z?ovM?59t-C3w&$0lOhScgKj-}4DUQ$v0zY`xo zpNSa{>WEg`*Nk2%(Xia zi#sl8)5H{RY1lwbY`#huboh#!5^|s5e&ZM64n36=rkKR#=VW{ArAw3Hm27!Jr!qKb ztD+`y7;QyhI)73kgcVjy=3J>`KC6_V7R1Yyw0}Y7$H1efKcmj0pHC9YA}UM}vN`1# zUl3iq%(87HnuA#|*D%x+&b8_g+1jKRm?av z8CA$e244Rc^^7dR*Vk4eL8a>KXkWDO%>_YX;0o<}WAAhvm}^`uhp2PzAyaJ_5O|mcG6$tf{#e!`{Xq> zHJS3w;{Hpy+UoCD(b3US0fUu_CmCKlO2@V|=#Fp5(b0^(9|pwC3tagmW* z5n$@BT0RcBoUsp^lwDY+E}ws8#Ul6pQ@rH%JEMq`5Qd)kP0r<-{nehdE=#jNgBGkC z-;I2Qxg@Hs!=zwG+4e+|2emiV)g@t6+*Dso^ZEQT4#FCbz{6FR%o%A<>UhKn=qyx3 z)mIC961YePJV_}`1#5p0A24kqM^&m4#qVeA#TPl?5_WJa6wj(zDTj=$11yh?lx*4Z z#AMOP$sPP>kSdSH`=wQJxM#@F@163{J__)?CCqk9lH&i@r2Cx%eFvU3uZO5xEqLJ^ zIsWfuFuDAMcxBEQg`13wp^KAsd;)^qy1Eaq$5#3>yO?f|=QGR@RB`j#eNSx7hl(AG z-H~VT`z(1?lCNvGtyMn6bwwt@s%-_?AG+adi#U`)sxKqK zS~8Pns%{za{RpUzoXuWJ%1oD;6naBfY@Zo(7(|VHBW_zvXzfk88)qQa>OiDaG3A+3 zs6(U>YQJq?vzEe&N6aplISsT(i+^*h0Ahqw_Y>~|UyiM1dDs9rC>RUBzHYFBH_YOY zJ3Y&t<0$bHIq;N#LC5XSwrh9aze#*6^oi{xbT&jC9FnV-XHZ7s+IoKFACy@@5O(@v zXfm-JtXwU1CK7WV$Uw(P7{`XR9TZ!D4F;FRElRD2`8p2DJgmNJy-Th zDSYZ|<@AtUpx}w0I8&yLge?BzbxjMWdaUxnTb&c+ki=={{b2$Iu&$WEVPH6JM42)F z7U`l3!%)Q7h;4dRauctVEXYklc+p)~rX7ojUq8#z;@<`9WRM)SMY7 zo8^WsO=NTwE0+;Su3TYfuW)H5&V(64lXKr%h0iBTm?_tfu>qRZc9BGYfO(7cSa<@; zCQlF!Vpl3XJzYghOS;^3SGK|co|HuOxK})v*9D}E6m+*x>%X^WXW`HAXv1c8 zu_h7MB2E2zg!lAB`7OFzgo;=CmZ>R=+zfgG9DOUKvkAjC#Az5#51pKeo-1O$;;3od zD1RJ1SV5GH7T=Wce8fs;`15R@0&?WjyW&G^Y*|Ira%md?TZ%{UZ@GePDTR36gj+z_ z9%V{L$;#K@5gL%wKML<8la+&!xEN8&KU=TzYCPuPm{{l@D2}!I`X1{St^Ms znZ}b1B5-a&UUVA?DaceH$yXdtU5!n**_Pi#d{fCWB_Q#wb1M1C7z@SH zhN*${J-ydeZjxO}4g&GL%*BHe66`@OXoHWJgmXVOn3O1ZA$b9mL7GAe92^CD+z+g} zA5m%+j=zWO6M;t50D{(lf{(T;TtgH;TwL|&$zi#=z2EZ2A%zD-fbbo>CxKFrV*6P@ z06gc5SG|BD%em?I-E%=r6Pik=e6SGc1pkes`xeQ_ zS!whlw3m#*3K3Vz#wC^36AD6dlHy+?#A0N9^tkG#Kufn=&l$Z|jihp(zw(FOqFTZKx-3es)#2Aj>myaCJ;6kUXlN!)UD4K7`)0v?_`6L(buaY1{0n|pF(JwJ zy_AKh*|MJR&fJL+BoF#_l3BnIhKQk%in6fW_+&tG!%+X)VGk1Xv}a%Xvnd6ed*_OU zr5zDtR#azeMwVjOKGa^7CX>;K#WY}kex8I(iy#LsT%$=LW+Ap4{i9YiXV5|5U?%Q; zAY2WZGQXE-p??(`?{|w6^6|=kiNYagFVQNeLw6R4lIl)S%M)?WE@m~(R`B`T4ZKkz zIXO9d7Z>SGpXwicw6VA`MDLyzyMqn5&Trl_7S+G8T(A4N`;+u!cTkQc0^F<95T!qz z74$lp!E|n6lVhS#zUagnRAK3&XgXWH`|+p7TttBAb#JK(y&FAknIK7?~If+?vwNsyiY2P-`!)sM1NrXQygngS+ zRX3Kr{H_rmI}G4wzhCD!%0(d!KYy0u19}Ojq+MS6$jQr-R?48M$QRvLvM8^2^g*p{ z$$g(6YnU#ZxDSan{)+SNs)jX=UXUN%p3eTTi(Q-aYL^j3T7XnvtWR=0lW0mSa*+(l z1B3-*f2f+CA9&f87_h?tTyFjVd%^HQJ%WFvDf{&bYCIQWebZy9p%V29KpGswf;v}R2He5Gnhinna`S+K30~3w+ZDN zl`R3^I?R#{UG&FMgl9=oYFR&}vV8)7(;hTVKCTe9Q+Z|H2-VimB<_2Zq1IZHiK9rc zD7Jmbbn7S zIo-)9;u*^~{MH^m_?m+@zm5e}B}(xK2s@AJ*Imn^)MHpB1hMLLc+GQO$aNE4n2`W9 zLz$n2H@ZIXUb>f8NHC$wq8eKmf9!n&doh7k8{_1xqb`snNvqOqN3er4_#o-idX*Hz z=qZNay&CPQ#diNajS;K;swisQKWB#G4JzOLOC?0IrBB)6JB8PY&of5$m(76(V=7iw zEJ&m|iyGr7oCnOq!((G>OY!jG&u)rT9(~}=sr4W$Xyb4woWTl(mWmbJT{k~bcxX@X zZ~H#$w}SL1?8}T6!(cEOD~&mC=3?FiWfL4yMtZ!CJLbIlBTHoPyN#yHpOcU|JA^9% z{hc>Zt4$GBDZvn3De(trt}2M6B^NK>)MmPwjS4SMzRZcq>pW7-yjc9P55v==cpFz=)PlD~`$Q2!&(c3%Ky7JF@TuJAYwd!n) zkiVP=%upQ$S*UjoGSjl|i^I2}KZ;s~rwnNOpPurV)+fN>Z{N!|aaiM=TeQd0SpYZV zyzaWT`Gn3K@4kKDy^d~?WNr=kIk!}|XI3lJFcB2jU*h^}LLy};d>Y9Cn0mk9khgTe z3II}s)T@VaUR&%RP?u*hPKCc<(J=2jKwy~{%Q53OF}cpWmDX;%Ro<4N%fUIL>yr_$ z^i)|!QSy%8^ISy}ei8eG;KxEuvPkGaIkW}*+y5mpD{qNrg08uUeX^*FR+Nhu9&*wY zBnkg&HuGka1t`yRQ@LOHHHudV3Yp>?a**fi=0WG|AxAP0Jl3gPVx!Df@Xp1X=MHtC iN^`>-@r%HFIRB;YYZVf#84mu3`BW9b3ZLa(1^*AmVspL# literal 0 HcmV?d00001 diff --git a/modules/web-console/frontend/public/images/page-landing-carousel-3.png b/modules/web-console/frontend/public/images/page-landing-carousel-3.png new file mode 100644 index 0000000000000000000000000000000000000000..91946c976f38fa32a7c46bfaa8b71cfebe6b24c3 GIT binary patch literal 70876 zcmaHSV|1lW({3`cGttDhZQC{{PA0ZBv29xu+qTV#ZEJ$FGtc|H-#Kf2>)b!G_wIC8 zSC_iGuDV0zWyRp2F`#>8@bty7;^&wfdrW=t2?U8NOKz6SkdZ#_MvsPvIUF=0^;U# zwbeJYFm}W@Fg7)}<{`Xn>mtNAH{u~wW09eku@y2lGZ%NaH&$|&RW@|DFyt^I)vzd+%|8EyZ3m(FMm{OOK#}~4(H^yh7Wu-A> zV5Y}sVD+rd%M z_Q~#u#KUUm9e#> zgfI^w;6JoR=0==M^c*Y(46Md9ENtw?G|Wu;1~dj7Y#cP~#!MzA%kiD_Kqm8|?jg95MRv>R?<7nexW@C#lB=lG7_~bJBhUV6vjTE1X`e)t3#`fmU z#zvy{Hdgq5CCzF6zi?s1Vrb05z{E4}1|<^C<`wr<*MZ*nlD-D0uYt zviWFg8ZmL8NhXQ`J!HadtE*$uGC54YoJhwItWZ@#2?VGH@Zjpj7?d%kO25)|=_9iu za757&pKb|5h@Iou9}w!aef2QqiNGG~knuR=b&W#p_&QWIJA!5RJkugCs7)15n;a9Y zHb2%Oo<0TPqm6L6%b1wM>8Xtnjx}ye8imfCo8V!P=sSJvcm=w>6QK95IpDWLW_`&T zAcTgsqlBFi|0ag7(y&^*vQw-F|gde`;86a8-Or zqwghPFxJd4@M>-LGOE%4?TFN5-5-oiyu+?>9W+rCFG34%Uk_OWSs!)wBy5TjI)PR5 zcz4Fbd5D_W!DUY)1~D~c7xj?kE8?)G8^s#$R2#r9+b9@4nVLr%w2MvUP6Qw32ZN^; zI9kAjmpr!ltHa*lc1EQ{T2)e^$>pumOB;<^L3Id|&6skX3@|0HXrxoTN- z6_QDI))7UyTMV{^^x7~(yvCzwtEU<&qhZpilT}^(;)}LLwqo*qn-kS?su7M}nW4we zU879CnVv}Y56=vrgZ)nTr8t_$aQpLT_K)$giJ51{fPLK@N_)b*K2HHADYz(95(al&&_{#nD1iB7HDBptemgrQ1E4<;JQ7czGzjn zuYaakr6OSUs?F&7b{J2?CT1EY)*|v-Ac_{{Uh<-9Pj2HV&J@elra8^xUGd9u zd)3f!VY6*Cf-6Jxo9Ume*@+sv1-EyHvo;GH+N%u1UJuX+brVO9T0;nztPbrc|4-*s z(1P(rCKgPDxytyGEBphhnzyGK*V}w@#by+|c*I{~5z;M{(;5d7*ZC^`!$F4jw zbnk0CyMNMYKep%ZQ%C#Qy~51S*DP6ibRXh2=3bv+JHMgcDp|lIB~@erF0nT|e}Mk^ zBvM^cGE?@MXjKS$lJ1G85U85JR`Rsk(+jR|Al^V2@k(Wnhs9ynUN-rUAb@~??9Rlj z=kv`-)EuScS(fp@SgC2~xI4B;AGqTCsgr6Uq&MJRhvD=9wmf=V;0_MDGi~}*X!~ky zTzmQ|3JdTqD%&iurj>N`Ik5NqF=NI(TPyhYynFW!(R_U%(Lqj}!G8AUFKAic)}2SH ztlpb^js&7Od1n+n0$j=~Tx&mKu_)N)g0DSsy`mOQL~(VY+$?!3(-h|eU#4|yQ6yG) zgq7r>`smcIlPuK-bIBo=U_*r>tzy9>Vjs+c$e3z&3!5R}f$H)5QL-)0>nSX6*G+JA zgV-r)pUlY2Hw8d*THDvTOQp*cOL2p&9k?BT+wDOQ2s;Pq@)TglF5rtm9J8a(9BW1; zdJwVV8Hy0KW&HA2)2M9eeU`w-$v%38v9ACe0bNQHC@{vnNCh9I19#X{3_3Nvv=ORG z_*|nZSCDYV&-5rXAn983EL^lu;-G7iErKQtvZW!wma@u_$K#%#pQqJr9UL44z_i+7 zL!+37*&Ry8`ERiVzXvYQU^F&rD6uN-XYF;&6kWrt~`Ovh_2Z2QG6f&lpu8 zsF35F{;}uPfVkWN2G%$w7PVyVD^UfP9zynN$k>m3C*I0)N-xiiA8H3~kU=Dh{~28qA%kKFKX{lM=p*D~-uHqXtQz>zbv8wcs_uPk z94h*6q!Mmd5-RDTr@@=@k$=%om z@rFV5a!kB|rc5&=k-bnkHeVeFI}r-f*S5{G3p%9c1fyDquZ#h$F<%bQERxKish|rg zKX7#K5V#yaMwFpV#)1WRWkV5x(kX4!-)eO>W=!338SSFnU>j;UHO)~@K^LzRB%BHi z*3K2#!+Vva1Q%rqh-lQ|Q&KFA97ZZ# zZdWKuN|uz=T1FaUCW&J#4J8sMMuMr$B}URm!O0+Jwlk~aipZ|Y_VMLYYe}^b8Oktq z@NF@idBBHBUNhk`{lNeu^2I6gYYVL%W#k7s@^o}lG~2X#ORR%~{q60oiZ!K;jg9VC z&OFf$UN$zh3RR)eUaKDsnmPP@e0=(e?xrT|h??NjkfW26jpE3C#^i~kYfUyjO-AOD zKgwohLh&F)BZKDS0whDlY|BvUzS6{AW7)U%3zAC;sP^h7D3gMYicpIu3Zs#$ZYgJp z`32+%NOX4*6OTLK0*~AT3^D0Bo1iHt3iJpVjM|=QnosEN%NFdc_U^9}eL-p*DugG_ zbal-12L``#4BN|6%xt_9ycby%6%0~5f&<@<)Ev4Ntq)C#7o~Z%FGg=jCCzNWqLt4S z%=HE5t?hbGLfqLu2fKkU6N$2vOj+Bw(NUx@wI3gu+%|`*Ohpb99UK}8=F_+6tejmN;D+luvg zB5k{$QGt-^fEz;|C)?akW$-#fnNlB0uy}@;jO69%rTY=mU`v2o23tPubQM0!QY6yQ z0B!FSrNlY_7~@qPj#l_WfBqYo$>X;R(g7V-= zz=Q}02;WK+#N$0wVTmH_db(&|^WxzB^YRE2+rrr7xp0*O^z&!B={@t6wgb#TcfWSy zP9;)pWhB_KCT7T#AzS9QXCf~T4%7R+XUx>ncg2)Xk4rGrgK}Wo>sD6Gu9|6tPNvoe zo{=x1DxIwWUk7gQVFM?GaA%$P)_h5-6oYWV4@CIpek*-+9DY5w`K6c1{8p--X1>2++_*<1EAHZ8d;nf@)Sfx7E#9vw5-*C}Zt0m^nA|l4r zD*35ow%S!IG+C_IpPDwk9Y{;{`+{KBeBN6o?2KIMFpBW`^st|VlP7__fs6Cyiiyc? zp~_fJygVLzo#kz9gCB5qb}x0BZT-5cr>E%&NU_gX@`K}-C`32C-zv3RddU=(a#(5& ztRBg8+0gw|Ef&g9F)_>J$&rwd=agoL!-na|IHrul3wf4t5VR_ZKtfs!Ch7YKWqAck zCUtj9Xy?Bz!0o+%2Z6MCv|~3aKtY5E$mG5z7IVU#XyMz0zk+Ggwe!G-r*mJ2+Hart z!d_@$BY!jW$xgd&l9su)>sQu4phMoI8f2PYC|7f_-rT5nt6OcbaCna=43VjQr9k8( zCMKq$I^F2-*b(dTr)kXYGtWJ^=%Js8B+CW9tG@7T)3w=ZwW_P0b4(G27_H zm6d_XY=$4u$BGEs1-w;CRUjL3W%@NW8lU%iUrjfNk2eDW*cyw&-pI&*xM|Z3tGC$u zNuUmG!u{?Hc)J?L&-SQLP1o8O+!B&uTRVrdMX`sBq?7f1^qGA}%NO!hFE|P(jm3gA zgB%-G&qqZ-J0%hv%ObXxu$Viur!*HP;wb@%2|a%)nX1v(dtL-H#-%|>hyEYDLsnO1TU+ zsUuDCbk%mh+Ueu)qrh!$u^*_jTW5VH1C{*XaF1l$(B*<&D+S)NVp32A5R$uaDQLdk zmlJPp=Hhy3RpMbiIDXY2DlSi3Jpw|(jn)fcrkFKu@q7x>=ZIp3iULxs7qZldy0*5} z>R}8K671nqlRw!f78P2ntLb@YI>ajub}bq-3l+`3OJycdv&Rw1j*5m8$PM5Zow$^{ z{dfhbCvR<@tIUIht2ixE9^42oE~q7q#2Q#{R?wWBaNP-=B+9}7Zlj=}(3{G9oFT)5 z0JEEfY;(E9rT-43W^>{wg+@mpNm=6lb$gdg*0(#F_enma-FPl3C=yFlMC6(*C^)!p za7(~{{)bEx?(Yqc!{4h7LI#)XI<-b4JB3nNXV1hL4b~H@)z$7eoKED*-$soXc~6rn zw0c5EMnG?(~%yjOd?zkOopy33E~Gt!P6W9-$RNuh(X4`#2a zsj+Wqx5@6bB^3Vi;{_J!YYdpYP8fzvUfQCKI-WuA%%4Z(t+J9@*JZi^fu&4z*0u}m zZr!EKlI*Gs#|Uu$Ss8*vi8`jb@81$KH7kPPZ1nVUgQH6<1yrAtq#=4`sgHlP9Z62O zA5i3me~ecMp8Jw1m?4I>=>9tY4uU()1kys5N4IA}&0@aLY0@~xsgn`omXE(j!3&=p zLUCDQ4(U>|rD}(L3yzm!IQo@wKUJui-WBfZOAs6mbVUkZlieG{mD{p*i2yA5$`y%` ziHTrt@FMM{vLas;*B<8#WTrnH$>WlE0O$83VPCivsjcgP}Y@pd&5~cSFfIC@4rti!2_kCp4KM z7>p{9b3&t@__m09W`JVS9XyaN?2UY4uuR>FmamCfc0Oe~v2C z|13?gjr#gBQ z&T7+}o5|G_ibZEubXj5~$dbf~(6$ze2=HkS(T><$^U~;O1alVV>DA6&r}gC;t1vae z?|7JcoS1&!dpv)_DcQo&KdNJ*Q#sJg^M>`F1(XCiiFM(xU2%m?QSwfomhPb`*oIUU z6N@@^it!XaGOA3blYiLTef#-X%bXp>AK;n6tU6a|-N(k%saf;z#BMe z0$T`AExRq2^%mq^Q&WHWA`tP2adTraBn804=vMsXT`<$DEde&gj4FZ zKUf%C;Ts_WlzZJE9e9;cM5HWJq@!?op=(3G@%%wJ3qC*An*a&|x=%_^huMBm>~ub- z6n;O7kIt115a$1GY-%b*xRSaujIdCp=W};7^OJ;$owz`$ST=^*Gf1n++RrE$77Mz~ zmN1*b21TdIgl6fS?ErN-umB&1-RrIOz8wcC?^=VbxWM-v>*3Cp9juq&>jp1|Hdp@( zr@$?+6$s8^IAm3}A|B&HiUQ6CJYtsmUg&x1zT-@2L@M{TMugB-+r_Wo8kj-pn(BNK zB(qYZ!eWR(TLk*99Dd5!1$d~DDLV3Tx~*Vx_Bpb{J@OmLrXPxPMKb=vR?ah_6q=CR zEy)N0kp{_LX0WxYG+%;-EmrDt40%N44U1$v-g5jNv+%&JK;pA3?qUO4rlRBapBt$Q z{>&M9bQs9nch6^VcKDu%d-)-XkqJz-Ii2av>(EC2kbBe??O>vGQ?~bTQUly%2DliUrG!)Il1fE4tc$T!UF9MM-Cn$9u5{29rc8?H223zoSdm; zE*GKG1sFKEK*uMS5x06qX#oM?f!JSe<3Z2ffqI>|Ls$XTrRkv!L!u) z3>o0SBwZ!dKu=qX=UhvbI$chuXefrcnaRlr4I1y8T+T;ZAaiC*zcfIgFYk`$C<{*( z&$Yt4o|Ez@$PZmE);WCMJ)Kn--;lIB+*K%JarwMq2|e80b=qz1E;o3f9xs)M;m8Dk z9s;&u+eaW&Cni$xT3S`A5~1s=}lK}z^^;!61o-N7GA*=xIiEupQ>bh39Jhtsj7MBa*-(eo$u zQoIYU{T|zvWAoQYO_*4hvt?kH2Cjo{~|u_jOHu_u|EP*+)9O^)Ba zX>f#v2iJJ7`(BueiUu4>F!Lhq=~*#fJTIL)#fk|Bs{E1XxV5riA~h#L%;>0D!4BW0 zAg{h<0gjmHdLX^U_w@bQ&#`bIkT^H8RMD=l?#UcA@XtP6N_%)w3}n9_0_<|^6pZWF z8?c9+(#%6{Z`z6Z1ZDQUi;K!#&MVm0X~M!VJV7;qBiPllO5aeaSH0P78!;D^9MKv$nhz}pW(51hWEQiN&CS^l+~ zGFsVTZqq1jCo-rKs~Z2YS`i`qRhp2hWWZBFkc50b(Q2(;LPiQ1f#7JV%V$Uyr`YGZ z&~|0?kMYm3bUvcoD3>I<_&Pkk%vs&?1kN$m#xkT!B|K{=2HewS)?=~9*d&R+qf zkEG9A6xh0tKte*=Gtuf+5)f#*5O|Wa+T-o*y~FA^uP;~i^o5;^#xt+9H0$C5q1J#H zJORR6Hp9tc@ob}m4WFL>M7xUzT-1q$S_%}}k-j19-qisn8Z{j`DAauF8AHYQym`Iw zbv5#czIOe2P#eGf#x$@|%*k_X zwVM&#A-EU>PsH@Z&UELB3d`32jC|GvTAuv|8v0`3_wDt)07)OQrR#w_F}CEC_!6dL zH|y)&G`vY!z>`Rzhz7?>ud7y4ydBa{(n~VpD-hU-AZXfIeOcs6;QqptlYZf-CLEja zv~J=fr0>nV9e)OgHoLsvf<|=$Eul%b4By1NCoR|3%Ua% zHsZiAYR;IjYWzOtO}bF=`2fdHrsVPAXSb_cqZ^}qHb!vKK8(HbI%DwQ`2!k#*3y9N z)-iYTpUWorxFNnsFmWy*z@D>z(n%YyOh^;L;=IDyJU( zYf()Vo3U+e5X^eC+U;h)hXA>fx^9>7>Bc01oCf{ z)KjLgA~59YfUQBlYS0@XSNE&VbG+(gB4MFFxEF|X=>sfS zR+0lWUw+GcOAZaH@}3a7Q#O~%j3W-6T!oB4<3k>402PV%WzZ*S?@XoDWtcF@Wj6Gr%8fmH67bA6C6cJk&& zsmt6=JRe<8AC4HIQ4G7vGx+SH?xn~IS2p>}No_`GIP~847g3CX0rCf$9Ff6Lq~O(* zQ6%Y`5U#85CujnIH1WZ#f=63N6J#?jDeJr`yp29-ptB^N|5DMxAR#-cH~f|Krw0HN z0)Rpc%R^#eWNlPB4eO2sD)7%ol*PEs9%3r_2N(6(U!3*_;aX@ilZdl6!$77pTK-qq z#-yHse}?N?lIc257zq8^dz!iFjqv~(Q?Jtei>&%7$gIf1!t^4`k@Zbw{G>_!hY5gg z*^QFMtkgI3KP5d6+lY+N$fRNmW)(JgJ7M^@4YYAr-=-R8(fRL?XoNKtNLM>NSilp3 zIq3ZGD;X``s5Z|GURAUAA1U=p-taNrE0GV9j%x4+x;s9 zqB9Hs8fxjx$GdRL^ApbBsg%1Bl&XzXUG#@)kA#^|?wMcg;1nsZxCTGz+8Xuk>395S z!9PPUdYQqMzggPMpM0A+_A)c`Ax_+KiTITE_9!|RFpA2=S*j?}ivjP85wsv?Gp8~c zde`?)hGF-{kL+_dqYx#!fBYCFJB# zZB!^3rw64<#qfB%(ue`s*?QvQ!+_h%mf6Sq+v#FOoA=wx`noox&-?RXlhvB@;bbOY zdd$edP47SFXDfAxWYpA_fPMg==I!O)V!kByz8daROc9ie&Zw9=x*heu*9*-Ci^WNW z_|;!FZIVbSTgPOG|t5b-H<>P@rUikwatKc{xT|52se zF&Kft0r9%4R@=`NIFG+$|%h4K-L%QQ?-9oE@c7)3fGi(ERS&@>EHn zyQQV&dd=tc8e(dv`!O&ukl&J71|+qIwgj-hx-!Bx*;jm(^@EgTQl?ZSmZJ6!|O6&TmH*Ph{Qcg_)Ufexud~8xw8=Z5CKCe(xU#c z#GyZ5UqvIa_)cJ^Cd7zzg)|-!C=LUjL`cAXu8I)Nap(7jvP^f3*Q6 z8!n*{kc;W>7hbMZtJ-L`+jTWOs4*P6SZ|rD))!jl^?G40X#(`{TLoNaR{;w3-PZMa zy9eZ*1ertd5E12zWitU6Ho)D&NR_&KM z@CzL)YokKGsA4HfGWVw{#h?I19uC?jLHIWTI=ZFw8TH$XG!X7{Sa;lO$8g}@E!hSA z{rY{ufc1Rn!@1v9Ah~G`3T`JWnl)wTZ)Nu2Zn(wc$*DXnRUktB8u0%zx~PP zc0+1hYq0o+gEJ9qYGNWezTD;0)z!sE9}lV zMi#!nmxaBLYbu4wqwjq=(~{~FaG4w-ozEB(@1Qh)FnR1yjeiS>>lzfR;FDrSdZJow z`ntO~Wt0ItLO`HrBArd()!4`gh$ApK*u&lZ%Lo|6w`dgz<3XEmXlQ~)@;1|qTW$(D zoWv&(U<8oD!+h>fZ_kI>!LS%e=;)Ac*Pc*DP;P#n0iXm00FSaoVI06i`i#uRJ+X46 z60tdlmQsRg!r(=e9m;WYqs}uU(;3et0>tk781;rA?k3ohtwsleWltrXsF5BBI2%!d z7Hd*SC})7QY!bDG{r+g;tfO(QX||X1*>cVAHWxC()sS^l>kVaEgE8ht+k}2B*5eJ*+@yjLuhDet+#sv`QGm0pJ?i> z)*j;JcrlpG5ci_SN_dVgv!2YK(_m^Fl{VdU-QEA3`(A*oExgI`hU|4M;n-gCw0op* z(W0WGKWq(S%@s8w|1nY}ISELhPqiiDWH!OpyyrTZM>}5&affA^!ZV>R?DI&TS<7b6 zX??yu?0SDWH7FB0!0o(0U3xrQap?I8_A1XEDSEr8Wdn!9sVwc}J3l{pB1~DQgwvwQ>`G^|6s zCon^4x%}eINC+-xf{n0Wp1fetnvHxM`SsGGEE+8=4wfvhTKr?$qAGu?OH>9k$I4>* zPj>o>tgtIrW6sen8pSp1AV4cVFUQ4#_KU{8;S1YVecZsd!JR^g@w8gN_TnsWoxKV_1s(3UO}Vx*$N#gsTvF;_YokQ1s@Br9|#1i)vDvccq%Z6Bt&tQMqN>L_0!8WZg<0c#k3o2Falog>&K@Oy3nIMeD|{UwCx%?H*eHV zBmT4d+I$YL*+fGZhkK?ZLj#9L_~iFD3(fO81MqMf*KCFcFtEQQGQA!bJ6?bOS~LA< ziA~VwRd;=Sz4nJd$ZX4eMYPtdyC}9=Vs1FBeBPZXhKP7FTSN#icJ69Ac#}Cq95dRC z?6BjicxqhWs8OpbE|m&%@jgl26@$m+aiUS`Yr?R<(3Tsg8I5RjKD){Mf)k&dqFR8O zmetC2D2K=C_|$*CH#?P2f8QHQ1o6S)%|MI1kJ)ZX4+ zP_Vb7+2hLP@Zmfqonfg;ughSt<@PXt0_eI|73!kDX8T-&SU(mkd|m3DTfJcGix+0? z+N1r9GtT$leHz3pc!{+*;QZ+8CV&T;FnM`v119q)bcm<#~XR+^odHm7Oh zP2Li_2RNV4=ePZ{+)R6`8_&}g?(d;|OegLg4Obtk+MKLsn^l-T9zghX?GlV-y13V@ zjz;ISe`;K}6~n>&LR~F!i^t8^#pSqpc3~T;vs@qYYID|dPnddKEtAu7Npqx@sOqXM zbXoF79)hcqiHlzp+4YB81$(ki1KX9T~H`KA|L338ZCxAj_hiSu?m@jg2Z?u5DYh)usO4=YTe{^B@A zuuBSSc@~=WOwRjfGGPM*4M%Qol3F_A{o;N#HnKNOG%v}&^X-;hjUQRI(yNOOW2II^ zS;?)O0PCHjN}qwA5~Ap)n>}EL7ZyI34uO-3UGt3}T}O5%W4^@M0$(^Fw`(cCB+BGa zI@8wNB)$m6cql)2DyLL0uajqbwJ_ugDDyj-Ye0h)D+R?dV9S2JsMhc6?{vg z-)r(K^gRF@Qk6G1P%I3B(O*ig9!)X;U9on+fmdJ4;g;9S8g$ z&s~5i740EQ#hgkrEJ^bz z#E`~S@750igYQ>A8VJ<2bk8DIy??dujwjSP1c(C`Wr&j{xpg&%3hnV; zi?X^1jCriKzFne!Ua6N;Zc|lY99Uu1(F}F~-{p?`j#>|T%t-g%Hb;M!NJshrT)HuT zxWBpRX_Al%Lrq8|iOFdnraUY`-b>m)-EhLgaXt5KmH#%+Y-TXt+ncA2x#BVxAG5(` z)A=#X5K1HSixB1Idv1NTCJ#}k7av|+QrgxHq@W0t6+@bG3<_LDWj)al-9Ub&VK(H0 zDt;3gCyUNO$Sz=e~+6T`mreU$gg-@$i|Qtwy2z<$4{{yL7jeFMff{RMeqE9XO8uP z*Gq5CpE$x=F)V$^S5yR3Skla)xJV%1k4;6nAEAXf%$UdG%fdQke`Yp{tSLPu6tR&g z^f?kSvpk|m`~~r5!5rae1^MI6tIMDuY`lWn5e7eFA}0d21S9B(q_II(Xp1p5<(`%p zH58tPL@GJ&FH;mn^YA{tE;nx)?MjJE)UypegRP>LxmGBgzJO``L~)gSq$ujQBT92d zfj+_Qi(p2D@TTHP=_oC3)04?6D3MGPo2istvLWSQzm>_+ICHB)#xOtxx`YwAK5myYMrwg%JlEMf83RU_iC723-KH=oo}x0o*hh`?Yq!fagB&Y*<8 z!*>`#J1x6LfYPm{Iiv3@=}2i+yM>COBg4>ieahUI9HmO9p{i3=T1+Ei`HNDCwUT;6 zgaw_!a$9)8wBskgbDkdVT@Z{(i(Fe{yOqH~ej4b*J2N#Aaf}kB66)yBNlRN(RJ>j#ytjq%ZWcEQhLwA8|WQO!<^LwUb$eY+UMDt_0vy|4FH zIy`^1UhMvkq~?SEL9u*ML&YnKO6M*=YfEi0LXF`ke*sI20LEr*Cj>+L>L1hia*;-?CZx+kce+}P^&%n;UJT?}-$z!(@0Ja7w z^C_F9)1yp;_J`X7X0_R9_t2QH&|)JeCnqEnY_1;4YX(EGS{uLhkvg#9O#_sth_}BT z*6u^tTQtDk-bR_NvptTdIr*%QP>)AP>(-ge`S~apprG1ytPl62NiiT>P?`@VSn;Qr z!_46NS42eA*R@3t#w8?h4G;-OD;L9SON~f(c|B+Q$PG!MAT{1;Gbzh8 zmQ;klJDtv~1)}I6x*SaLMHQhamo>&OV9ewe4wZ2@e6Yu)6i-G*KYR4l@2bsb%N&Ik z*r)38-?iKpSN@)oh~8{`OMx%ZvJrllNp}N%fB4fF@4_S=PfO~4IjOW1rVbqvMSdDR z?lr(zH5i#S{JZ7SN6g^HQChaQFw+Bpj0tRw7jMn7|4%Sd{k^8k@$X`g*5m}Of<7m_ zK_7*+SYq?|D^NPrtgAB(MmW(TXxer=klO)d@ysCBN+GQe;Y`k4yL^t=rgyRVIvDp; zWTR0AEf%OrF#ckp%SbUstT~3$`xiix25?fwX14`${s0sfc>Q^}KUsjW#)f};y`B0- zr~3M*io(`|YQS9KuXcveiPuVKlqxdi>G(O9tQ71)>H_y`{ z%>E9OxPona0>R-x28-iG^3r{;FkH<`Ph-BtD};fJjH4v80pXd3_?MnDyc4dc7IJfHd0cbboE)Eu>k?E`Q^r(j(=|k{2MEtL&<3#E~1%naO z>IE!R6}LJI)g6!>uIhX#`#ui(0(^dd+4TVc9Cb6!M-!ZcGjbj4o2e$y1yS^YeWcAxQIX*D^$gnW?H1C>GIg7S#ww2fz8KvvBGD{wGFrfkrT3xP}TT+ZN`#Zc(Y2}ZF z2PmioI1WOIh(+3vDfCdJ$*u_rAW<} znC77OdaOykR4%o6TsbZ6hcls zSV5vhpiYnvaqZYCqswuj8fRV z=}~&Bar5jVCS*LB9NOS`s6vyqjy}3CNaI?o(`m?8qL`J_{-5V@@$s8(hdA=)g`_-j z(&FMFk&%Gys!BA!#?8kTlAWEN9??eu)tA=zV*@6wo0NyBWLJtey_)T@1&r#@<+iGE zDIK%JX`A6*3t10VX(x&qcH9{u_8j*JqipWgEddaVkVeLK0CI=%7(Z~WnIVR zn*3z_n_F}KKpo4y#%tr&&Q#Ta;dbE-AcFWGAUpfLyOn|L?I)(DBGzN_ z;W6{^n0!5?Q-=){ydK1|2{Bdk8wAbRD3nSDWY!w2))see3KWl*YYYYU^vlZWnK_!G zD^hrj&zR`iIZtghPiHPCk9PDjtJtdF=5C831_4|Bb#;$9mu|U+1sAg6&d}QR>ZBPn z{Q+X<>0~#fiup)A&@g)gW?_L{dHY7B5JXh|M0$)E(90uTy(Ow9J5CG5z)Ym*?(tl) zgMG6=BcT4ho3l3;aJ$Unbb=t>P4*ae++>T{|VICwGnKg~^!oPC!Qy^NYbPcbc@D%LAc~Ja|zm@b@9u|-&MGVvYlm3|sbIO?%kd|yRx_SFAVg_kTgn0IB6o)dB zg1*K%)t+Wi+6V(Df=T0@_NmNhvYbXT$QZ5=;&2~xOmC=VRvVNLN#YQMTr*-ny)yPM zuh+o=kkteMw^-{M#t1!?%0FZ=dP`@8pM*SrcrLzH4&X`Dq6{E`evH-N@C{D77i7v< znR=)m1qk;WWPUE^S{(#3`h#e~@e6GKE#j2bm+Vk;DAoYP_hd49zu5qtaI;s1j?Ozo z8es4p)~#eYY<3uLjifbSy=&AYZF=Agh~TYFMeDk{ooJGt-RpkK5!&7=C-w~BfT7weF*Rno8o45%-6I8TkiUo^cM?#baGY>WH4#U#2ycJ8LOYD+ z@FzyP)-Dp8L#_&W*c7P8E-9tHYI#dtlTb@i95N9nj!j5xJej#qltLHE^`leVI(e&G zixCQd$wlBD?q`;QR$I}dZ;>1B5u!&{TmAfadU9tsV(t}HG$8!?Rlk>t_!IWzt~4@5 zljx<>N0SS94&D+9*`c??r{L5Ki?g-Ewqt$@Mk*xQKvVSXV%%=5EM}f=-xgofxLdOm z)%WKZ0KaHbQ+mgBtt`zA|J$53prNH(w zUA?$A$dC(gFO!aUcl$KcmHx32vEI57A9(Sab=x6|J zbE|KsJMOsGz4~)B;MF<7_{r|CY8t=%@O7_?g%9=PKHpmHE&=7XJd9Rs-Qke?b&DNA zJ46TmNhu$^$DTG3dWHB}oARIGKgIM%95I&TLjQUIg)2&uG+D8LenpR)Yz)e5{A`z-K3<^6r0zOl}9#Id?FGq929Dq;FBX#`)f8Jojp zggFDp7FW=pIp^a3z+IG|z6qOrP5yF@Jzi)a2k`0mk)3;lPjSCK|X? zamkT})BpY($&_sxk*D$$FU&wU;>@=Td#9Pb6aG#?yE`+1eW1P#Rm~ws>X&T)7fN>p zQa0|&B)MLF{3kMWmrRm37_BNf{gf;6|8ERXb^$=+a@$b4l*Ytly5PWnrDIa_|AZk) zfrV&Vx&;SfzCrk^|HKH8hD<~C@j}H1L^QX*Jg@|u1L6V>K*awF1N{?w;amw0UgG~$ z1>hB8?$(Aew=4&HM~?p>@rfZnnw(oEQv3d{o4$-`G#eFA^z_w&%Nxt$ZxD=-?8&b- zT0O1VRdZ;#QrvF9!L|QKAw&(Ck%q>F>8_w(8jTD-#ji|=dyfejWkfo{g#B+5BW|W# z5sP$VK{`OU4LE{6C(r%$N3HO;f^XD+G9|3_-LFK!~L+&V7)VX+Ey@V?(07b7iic?Gl#YHll$BOv1 zJ0ucA9G~q@OCTiZu~3djZL0CDISgN%mF`$t5RKy29OWyqu?$PX)z*RqBl1Oo5r(oK z-09PfQ~Iagf9>l8CP~ktrG^fzsid144L2h@yjgoTCKVprXMH1xrC(~2_5RZ^l3giD zN|Im^twuPb_qy2+Rn^6_yTrV@AY9fHA1_Xv8ocDidGUM**g27jFM9|?LU$512dnW zlR+~&3qO%P!}Lc{F&78k&zEUaW85TAskX6ZC!;*1^tA)Oxb0*M7-o`1&}$AK1i%+RzF3IUC3NLa2-J(1hm z2KV>65Ecd^V0zisynpqOZby8mZnh?>0P z$W7L6yJns{*HURogOqKBzMm|B{#-PmVD1LNPqn&3aj`g<)uc0isiZYpY&el2G<6gi z-)AaEB$$$0rQ#OVZk30l^D8s^s`&ZndR(`ssCZ%7oRA`c$J20jvgbGLMO2sR3L_*^ zjXu!aAq%K8x6(=|lrpo`dF_`@!JJ22s9dMljy8unG?$&xW(>8bTPq7y$x#qNOT`~U zAs>Dg?jL4WX`MV_dv1j?yMr*I;X~!%5yEm`p}&S z7YR*@1>=D4?cO)qj)@uE@NJd7qP zPvKmSvjx&`p9%)`uyVeNFkf5`oj$0md!QeaqMinM_7w@Y9gOvHjok^q@F~_VB-S z58R|Wl(4KlT7FeN42R-e&(7!+Ebq+RyT}K+1QT9u`(XNWNdS0~1CO<2H)E!nLGo&LyW#iloux91_H6qU)LU z3+cl{MT0;s1|7M5X8<+7<@Eu+OOrPxUR2T6dPeJp0c~8M_j!#=mmF5F+wpuZkBw9h zTiF$eRGU;*2{m zQTbB9`tDF8=ZEJUp>CV+?!udBaIfu^$zk(%b2);X9>AjMnayR-%QYH|P|%<52oI6R zQ%Ch)+(=xf7dM?mP4Z&UdWxm#B@pSLXLiM)mKWU}cyO}4k5n4$vGq;H=@T)9##mry z@BK0DW%u%6cY`MJQM=W6DlxzN;M^RhH`0HKDJ<-E!yd34Vt+g>tmWq6!DY3{PFGhK z2-HCU0Pt8Gfe%9}0a{8OuyW0C@A@Aa9}hfT-2J)Jz^~^&e&7`@WbARce?+I0X2!l& z$(BLfKaq%sCd%94Q1J+v>odSPSxpb{P;?pD#S3WH21N#XRL*`6YCDFTwBl2&!cBam&%mSaKD5E~aepHpX=BefF}PkQ zE=lLpEL&fuW4>#K?9uSe`N=OHurtONDEQAI8siWUCQ%6uF_T?evF%LNayNIM&o+U+>jnshV>in4fe_l{%<&e@J9Z35=0`8rSM^rbiI;DQbOo+-_~Qlp2cwWyYBZ zH~1efp>(nmVNIM!$Nk)|F|N@p$9e{3dUKOL^a&=sJIYLaUPH~>J>ATa^fKlb{%%dD z@toMsiAx-<#eMp4;qexzDb>H>6)ZjS+TfYmFpzX9>qilt+F8DCcYS+lK!5^lX#$E? z%Rjv=5$kAt=ykZtm$o!g7SS*~^cS$w29stVf=s9P4h8*y*D8+7foFeNHSje`O|0iV zD#O9Ie!0bJ1E1$xJ6>L1Q;ueRVgiEay(ATelrc`5$*%`c`*2NZNl7==_*Mo?Tw}cf zzL(m^bGdIK`_%CA0&Sl2O_wbw1VZ)s6Upc=V41@(!MXr4(Mn^;Wp?l4+wDz7wHi$G zHEb7KTXj}r9j08SpY@Vm97z}j2qD?yvbFH)yrVhp)`PRi@XcbaCbRw88!Gb71>Nz} zfn~GJ`|9J=en1E@Y|PP|y4mmEtIG&T^^&Mg*G}iOwADzr2MI!#bJ1j6-5;?ly)QXc zO0^*>Er`Xp{`%1uBrgGTGe{Kz%#m?;SUcLjdzqKPk3lwO8J$zSoel92V*@28RKRD-;XNwat#+e!MQrTt+n3J{_xq6Zm9aYWl&u(3#J{EnC{W~%lLhjxhnY;sC^ni1mtFdsw4dkb{SkZ3oT$0C z3K#)*{z3$Xa9ANR_s|Lvy??J}r%Y-K!+ol%FDrAq9DNR&gA|mz>hl8F#+16m7b}SK z&)zP6(ucK)NX&C`IZVi|?W7xhtY@rN8C?aC+1BEUJh`ndqhYtMt&Bg8y>2F4T{Cmr z=aI2~G?=Xw8|H5DXyAMiE@>8h_;m7u@?xn5INRQ=IZ!#>dfRe=922*+wDk4$4Giq+ zGcD67odjQu$c}K4<`x!*6F`sGM>kWcX>VrzraGq0zoZa@mwu;Xf99f2P@7Z8LB`K+ zY7TZ}G3qM8HwzOn58Z3ed$mt52d~n(QhUQR6fJJv0CPZUDj9YbF1*=1P3?!XUMUY0 zq5q8IY{CMPx12#$Sz-6xdH7VUKMXJBtTm~yr^kUxl7U=YE%1HP#f_QZ`EvPX=&J2_ zd=p+c_;IN=SeOKFYxC~_t?@wzcD`?>vXfEofw`j8q4rt#sl$?CR;%Oi-;0oX(cssa z%Sg*Vt9B|Zy6zMnmlF`c(N+&CO;doOb-s4NAN?*{nN0r5Z`L$cmc_Jq)BA+<-xd>d zd-P483&y8Ho|#1_{VwA=!`8l0ITDf*itbnXjFbgGIj^czdJ+r#PHIsWPHMTVylS4k zpu%+2^fs5sN&Vf~Wp?ojn;V3SO(pog-*#Ne(Vr*K2ERh&4EUf%0yvWJ>g-r%u`<_c7bWsN;I*7WgJX7l!`uWM-P|3fs`T825)8 ztz!Jh9T19mLDJ!SZ%w}9HZMjdC%Zg0%Er3hw zH!_T)x_*AQH-dV*tk*eCC>eS-iC@dv+5`xccdrZx<&-l}!m%BVukM^z%h-Y@ro4RY zcgTi<;)*CSn5kbcOkK!*zuQsjt$xkZDMMGE=b+0otINbiQK#(A<3f+Vzno^48WJNw zD>kboxge(~uh})G2|-)B}xVU{Yq~jqPTAajQ)aL3UThcSF1|bC|-J zWw5*xpVRtlI+WQ`i<3c~039t@4NB*_TxTXNElm((4R-(i<0=cIut~BY8(PK7cw%9@ z+wTceKMmRry+M;k6hbpsTBip?yt=P%vzIu3{HPyiT?A3N3Ap!z-0{+14TjN?h1AsOP{ssvwOXzZ`MKS^gPYmf zoW`*2kVebxS0gvIK|(fH{VoTA0`>z3Ai~RioOVWi6~j8qzShma=h5$#f^miuP9hiC zr6-FuFYAxxlxwpD&6!e%%K7~PvORps%U-K0o2;)FX`jE9`}_f+rwugtTb#>4jRf7T ztnZ7)eeU*mmxT?vFcGOYo;g`yJqYeJKc2gQ@!{a2A`y)WG0Q&>-@rvmu<6`QHEr!+xTk25X-$v!W}8iqY;(F3Nd!prz8bfcsnCqAM#iy zcu$0tMxRpd1Rm?*B*Y~sbg)#tUTbP=S(WC<-kI=Brs8|5VOb33c2vs`9;ogUJ%Xs z`lO=E6_CjRROT?8^#vi_Y%n{LeXE1d-_K7K=&=^-6^u5O9z3|Kk!5;+e~)Dj1JPj- zogE)o&ys7_+Eua};7&cdt`2=sx%a=kc@A-Y&Jb(`6LE~s0gle}Yd{8(7p)XM4g}k= z!|XPqZbq%+L!6g`m)hzAYE@o54-Za{>vgS$?o8B!&$dUhph~?kz?4P#7$hno6f- z45i33w_rswE|c!3e&u{susM-lK%fK6-(Bgbsja}X%)BQs6pIsmhbNbeL~V9dvntBs z^VzID#jFkpZ@QTI80x7E71@{*em}Oxu)K{W;IqzKS$>X|J!d0rsrzNR^UV&ez`!ye zBD4YJ;OaYv^O#CfpV*`9PotjZr6DN7`?Jh$Z?#H2!i}p??Vlz|5L0ngtV>6>?SO^B zq_&|9xxy#nG-4C^(`GcfU712OD)Eza38?(!5CCTqzY_c3402pzUdV4POrK~lvDo_1 zBO~q;hWGMQ>DuZs|MC;Ycc>OH>D~>=o;u5os01|HY%~*&cpoVlO3GM3=Chy^8hgSv zN4ck6!+hT>>E3lb_*MBcVS%zp?6nLfJLJmgV_wHWAWOK#;-+T*MTczw=i}L0Aw@Y3 z;WLz^<)|kpLsvRgQC)%^q29;)vRFPYZ`wKx66&q1yIWZJZGf=F5r-y?A}%xdl~oN1 zeL=?g`h>_OgU~`-JzstgBe9AF^x#;|DJ)KDWL_e#LQnqz`wmb2;j++|o=Nji7eYFr z*Z#f))`(ZH)ZLf+_sNy|a5%-MQI+XpRubMneKQGuOL@r!GxOfJcDJXAw9LG*bWxZ+ zk?A$CoZ8HQCJEFX05+1eGwRf=gRK-3K+ey^0?;A9zJmia0`AZ1?OWx_CIlHb{jFZF zyQA_{6;B2yuvBC&i=gQkRB zIZ(;pj*kowfIlUoG2TIsx44xJ9lh7*598k!uMAxuTh{-8IO$|}Nf+}y;$qdB8=5yr z+-+bRGQX-=La1#&9k0FfRmjjiy!OtFc%gK?VRK05^{`P2XV%*`Ci4TikBW1uDFd%n zoJ8GFn(nK+MSC8Os|?Eh-2BQ16?qx7oM*=wRPRP$ zxfNx(nu}$7!VUE7ksduQ4dXV;Pnq2xtUbP9F9cXFjyw3nkfqoE?gZ=z$T#e3k)oJ6 zy;F_N7w7Q1w#3+rvV{(ncPS!l$+;c~uf<^^!hP#iWj)v^PL{7#K0Cd{`P=`9&%5!y z6{E8nfu`fK|B%yl6jcV+^GkhcK5MkgNb7xcPXs^Od@I)U${}8S?QVDJY3M~SR07P_ zAxl2B6vdF$(1`sU=YpXx4BKa(5+>NrLGf!Lh7 zAvqi>6w|W2uCG}XLie-iqDh`r+30b@VM&mB{d+~O}^ZVa-?~hl(v;pHBom%H^mOq3`9A>I{w>eiN@A@oI}?FM;5zS4QO7&9A4Z$8+$p ziwKFnbV~AmrP{=u?-H8T^PIsodX3k%<|K)JCwr5s^zqSEA9En>tj^?M@6)v00Ogmz zg$EmSyzkek)Z?sZN+Q{!zYY1zC70Al)V#-sPK_5S$VS&pk#< zVC+XjHJO(U=%o9hH%b^-rL$1JD_MzBgUUddncmEpn#<=={57phtTs_UYmh0 zjdlJumzEwb$WGH#s{^5?wp!!-xgYDtu}Qszv~JI~qxPqt2)~i9A!iswHZ~pVDjy44 z+#QDY!VKs&k+8iS=FRbPaP!n)63EEvL2X`Dw>zt)T@YfEG)rc=&)Ev{dZv6u_xMs3 z5L5TC<0M2|t6OvHy~`(ExPl}SC+N>_wALLg*SPFTKhm~z$@cO_LqgRdT^E;GSAgR( z1aHjzeE!Pwxbr7K2DtFpfl?UA^oJ9k9Z#{mrE=EnvFz^LINB-ljKJw+U#WN9jOm0M zDawm=6l9f}lPpx=!Mmm;8)t3!u{R|s>g8~yo%Zc23LL5f5)IaUWuPwqGp6bBOC3^@ zGGL6VD#2`Mnri?bFmfdJr@@v|N5CWs4Vqh$FhLF@daw!4L#`ofrSkXRsstI6n_-St z`#s&JxDf1RhGt@Fs;oEvle+9hz^rO~dd|Ra$1LMq5tjK-h=MPjy0veS7%tg*+*W%$ zNWdPp&R@TNG*+X)%D2dP`b5#U;K73Y`LRPuV%RaN3K%xr=;*^Gnc;gw9YkH&A8E!cmp`|i9Tf(RYE&&;ev^k?tzP~ogA!}}I{T+CEn6P-y% z&Lih8^HBzp6^CfxMm_EqnoR9N`GPfTw3Ni?KHxCkbuw@62cdlF*Hr+$Khk4Y;3}9l z1eNhUjuSKsbQ2xBmuI`I#X^)pB60`H`3gsb&vRzQIDEyV$KyKkon>vPNu&TzqcW(& z2zXE+HJ%lfm6F`G_X%d^+iTLrB6dMpd*&*MhsiAU)$6|VsmNU}zTdMX5K+oqW;?|l zNVn+Gex3x%`kmZg^3u!o|-`XL*T98CG}vE|JpC?hs?vpsV{jpvkIo zppfaJL(dun3+WHM!>8wWZq}FPiE}no*cw;zq~-f|^86g`LzLz8wUHy#<{_9ZZ zxbl3OmA)93SP|;q!)?9%i%$?%-nYE5P{sB8@aDy_0Z@@fvKyjkPeMXsXLnbNA?3%9 zAK(P>)$jA_#Zmb)3|g z^NwB905;3bR@!not$x?h>+-#_Ft$31=Ot|e^oQy2`~H1^tG$l!O$n3ZP3s+N1LRme z#v|&q2fQ4F>8y0tCfbM_!7@p*FDL1Fe#usjv{YRY0tO-wxwM>5fQz@$p3t2DVv~gZ zB&3YNG6uN2k0@sw+m>Th6S9!amCr7qQn}+>T_?_zq;ml%Qd~V#m01(?mg(YAO_feb zv1*N25=Mv8BO4TexViafFwv~~O$JZR;|NiT)?-tK-t->o_JF`UQvzB`2{ztRmGq8d zvB2L2*}=D{*4zZuGh%wH`eq=#y6m zWhguTBu3>N@Yrbstq$X(suQjDuN$IzKAnH_uBk)MuCq3VK~hE5_KmYz)5=f)$8k=i zX;Ts1Ofz*t8W_8Klq?`SGBYKt&V!X}pDU@7cj;Xt#Y5jQ}W9REX?SGXBbw zY>kx40K_th)ks_;&G)gnhp*47$2KuJU>F;&cp%<}>4=Tph0L<2d=h6!DU!d!8~1!- z?7k`8i&8|KVSf6hQAfkzSfv?A$jG=0$GbRDY09R%kTT~@ZFB}$$l=RgFZ%mVsmAcJ zO8rV|z=#a`JaTO)VltnEM$&M|ske2&uAM$$%9fr+Tac^&3Wxk>ri0*; znR>qB!ylk`Rt%p53N$Bny+2aGuK-0qDA?&?!}?6uE;g@@&^sH<{-}OgM+GlnOSgwW zP&pc9a#%BC!$XF>QS{vq-~DXxr?~)VxBAC%Z95v=2v3o_@tj0CVr757w_k?j-+J?!wmB1%?QygwUsW*t6OICh%BwYw~GpxK?_a~Dx)`9`SZ~9Q*aIRlo zZ%j=_Dc+>g?m{}pIy=+E1KT?{uGKA1z@+dKCsy>ANRSkTXONCG@W?NfR zUo?{0bI#^Z^M@MpzhsPQ``t8fKUOChN?=QwsR3xvGh6#6{go=AzY4T(nTle6QXnF) z!v)i_3MWJA`Pa+W#rfzXK%{cGsw+EhvzHc^#)KB|=MT3a%+6E%B0BraWNfYRfUFo<^-)H=n$vQ_BwSRj{MAa8Vh zob1f>S%;a|rH71PWG*+4#U08znxv4z{?0}8oXYMbo|~Y^+v>vmqnol{Fnk#wm{Il@ zKC$4azf*_RL~$%t&{tY&{=P}0?hNUUOBG{#dOqL5QnKfz4P{!z%66RE@>_$hqOtMvgKYZ zIXe1#jd#)CCRntSkLZr-pU^Z`?xPVX7C1ylUA}s7qy{;=IqC#Q_&rmL1_Ad}VJ9CVBm7p!OxO|rb~S?vbzaNC!^KgHVvb_br^?-FV)4<2 z_Y6_1#@n0oO_(rrOZB^Qmr|B)A(yp9+mui0?em5QLJnMh4l|upNf}7dHiP%lWBao^ zvt~buep}7%^v3T6EECG057I_z@Va!PxEIZiVG(x}guvJ^)}=`FpkDLUHGW;?86Zw^ zsI`)8Aka;kmlQp^rO${xtpR0W;O`Yx*&%+u+oB$>x>Kb@@#EQ z%c=k|T2NpQnaV>!FD_^K+mV4kwHQCDy##tgj4GH5oA_zFy30o|`LnvPH$TZ^tMy z=b;Su1)pT+!Zi!V+urByf6Tl+FE#Jm#xmK-HFESC*0fn^tp_X2?t6;6>wRX~#5fY19b3*~o!;~%Coh}o?lEwb97~u)5rg*tma#9^QFdiG6i<|_>r$LAF z{y=QR_^HM1cH6T)Hp9A{LRQg%m=X~b+#ELj5YM+xJ{{^NQ6%mZ)xL`k{6)G-aMoZOK`QF<{`=c%ZjZ4)-#gC07~z-O zl*2o(q|`TMeO;>3YNr{Cor%%&s)}@=c>Lg4(s6t-qq}0t7jiLFn730U_!czb0c03w zRJnyWm*CgGm@u-X+(%fayH7#x4OvWg=2x#ZNP20RorOy{cR~*F9lJ;4f*;OAoUr8r zi?&!*U9`#>ScwD_><2k(*H?m@IbQBf9l6gIwKH=ipY5vnI~_*Ha=DDXMw8b{-K$`| z;hF#WkVLCdj@w+X8ffA>F4z1{Yjh)*XXGPeedDs)ZO7VHLnD(oIJJT- zo?i+B28+-2*$-7=T`mVf9jSVz4;Q>9xAt2e{>Sqw)gy1sbsG08qN166msJtWDJk_n zzuG?2eh!uF4Sbv#%S`olWp#5}O8Bwi$2h2NN_vz+=BH!3cR;KFg#tBpZHyCv#X{}F zJ^dr#v7;`OFYD7RntK1I&b@I3PTM}r^gYWWyK(;&cux|~7Myn;0R89^T0|t1pmFfs zH849rjHTzBCh@=(juPxtI$HXPV(8V`BsJ-mwlp^{Hp_`MeoN4LLc-Xi-=$M_$KCFM zHUP9D+?bjZY%{bIJ8@M{xwsfzw>>u(yT>50VelLzdGPxZ?yidBeaT42B2-I0b0o;G zPklxgmWPDudvcC0T&!xU5IaFHK)NamL z^}mp|Tv=DRG8I-Us1%9E=KogHI}?KvhpCNJELRBZzMPf?7lB1L{)16-N#A?y zs_DM@tO|RcajxADy|NuA3u^>ipEuem|SP z*Z(RhSW?rz=(Me4(CU<c(rN?BgCht2i!UGToo-4XIs5fQvEEGe{> zAT2Uw4+;$3=U$(mq$EK(e`&k~+zv_;wz|M83f12}3ddi0s`tpcp2nqBy?(h10A}_Y z`0_zD>9sam4&YX;U++GUdfvTmE;@~V0zd`DxLxvi_r-|@Es%2^zI1==Atev~`YVHP z*4FE{f9Bw|@pyk&y;)0;+w3>@T(o`)=gq7!%>29=_hBIN=lmd{kY1*u0lQ(Era45= zM;LccbLespqb;%LrSrw+?(5I~Kq^ANAJ#Z>aEVH%?ScAB^&vSC0kHAOF4*pf*(lkK z*3Af(Je-c04iePZk)840^@!iI+4fLO+6-y5oc?js!_zZUTzaN7qN z@);ob|Na9*$?xw!z)G164c|LxgA=oue7>IFsXiAc%zmnMZHNG^gOwM@1u;;^Q68V1 z7=J<_+^EzPgUxlcQaJj#@qN_P#6s`1qgsFqxv80r%XzgV!9n#wj+C&@wbctSGyCh5 z8_%GFmzSFM+*Oq0S@UPwKj)58 zD%R90kfvhIx#g9*?bhXynq=kb=^MdO^jtbJdBYOE(0q#CYN~h4l*orHCY)cv!N3e0 zUPoEnl}c#l5HnfybzvAE7`hqY|L{4vW#rFTMhBhpt%Z6dUARHymVy>%l+`6DI7w;A z>G-Ad8xG`nTT_KsW!E6*k^VY<$tIVN=&mkIjHf$4+s7IWQ6qSIj=#TCG*{h{FPGTj z&R91H9g4^{iCdBE=bp^qcOP+A^_4sxyG&hU7Z&`S1zOz5srA=2dWDpSD!_oERSCo> zk0Z9!6`VkA_TV^@{YZvQ)f{q5aO{rVHYT@vkQhFXeq%_PVm}-kA?_yF^%fe3Bu~k! z;gHOeks3`YL&rTz-}(j;`5?cId+UAeGQY5%H2;gcDMK$d%3w5bBFhIjg7q@-xw?sG z<5B?+Bh-s=UZ)A9&OZbi0zj#853h1?J?+-}Vz}G#1-W{Nq4su^*oLT$!r$Rl6RtZT z-V6wMjq-apf}T$`oDbokTgnd6JNWlI#I${R?Jev0$%pAHxM@b}@N5#jO&`owN5j}E zv=EcT(0ubW#SlyM>&X08>Au2D>38ZOhT|3X6Y7fR4au)V#wY*E>?Ar;3{|&=7<~~1 zzRn=jq9zGS#mfh0bK#)0>OmD2_I=}+B( zyx2zKSV~4klRs%LA&K~zm>5oTHGk3Aq0P>dOy;4*z^{gIO_p+X^tP=g8akJWs3aj@ zzC#x*L8hyb9*vH`2at$zQ*2UB?(Om$ZVgX$E1|Pb65+lRffg|+xkk1}bFi3SbeQhz z@7w5E+!``Cd20;5TWs!inRp=nJHs1Rn9HZHOO!7AkV1O9p!fUww;nFe?&&APhDWsY z(g(|o?ldhuU+hIOHk*i=1BWyu45ZGcrc>I$$`vRe0Mmjd zs4eK`x^ChY7`(2gr>ak?@PT8rd9={#kZJv;C!Ya-9G;>KzU?eiG*IZ{JCpmtXVL1+ z*@ms_R@z1wdOfWvm|tp7Vz}Ut)*a{6i|B;UEotR)Y^u>q6k!cZP4BBq1j@8IGf1eh z70<>(b^D*2lKu7N95Tiz<)q(QP;Qh1uRz6t0Gi!}sI^0;*XK5Y-z!s$+j~8ga&%~V zI?B1F_@V}hoLU-o30Ebe&|7~J!!+sc{8}$SypU|bR_6^H>Pei{O5EC9Hz4mp_L4qt zi-#+9LOd7l6_h09kzoNR7Z^N$cj)d+3o|6RgG^#Kf zV_WgIrE{{2mtDUo`Yb2j0(NzB9r?wU;U3Gr4+Q5LbwH^QvsKji-P`MPlyMDOBlMrAL5SI+@intw`sf=ZEh| zl^`|=_qQz@(pO7Iq5GHI)<4mEF7GZnh~o0!E2vo$VI{20Ll3{I4!7xrmKI0oY3Urm z!vsV~Nb~^aCO$lJCK^uHv~>6ui-+Q4JDr0BkNz-*tyMaIw4CB#3w?>NRWJ5w;cGhk zQ55dsrmfez@^l}4dsAh_?eM|MSP~l>hcGVWL)H0SmF>~x2}96L&m(1Rw`<;2clKt6 zaK1{Ye0~>wMr5W`4`~o_DS?M!-TQ7Iizj2avo0k_+NOF~0JC=>ehAx7XpC@pdAq`T zW`8775e#w`{-q>uD41P;j02->HEFfi2jUo;!wApdhJwKE@QJJ+2+pkg68N!Zw01ms zCUA($Ztwat(_;8%8>{HEX;z6mNRZHnhk)%%daY)VShmUUOlvxXpW>ONJSw7}7-(y! zzn{q2IVE)YoWxhyQz@od@$%Uk)2FzuDZsno58k3q%om_ z5O6Z98T=|@dz^*UjKh@aLv(-OvWuev;NO?DG~`oKS#Mq)j|d8wSwBOnSJu0-IhAHb z^+5A&Q8!6V$-0JbQF=1aU^ zP|Yj1rdOu<9n>??Y;g=J)+PfUmJJxKZCQDIW(+qy2HEWWNpVv^nYDNAukWFD9gL|! z9q-}DD&VT}zO606$@_q8J^P4G#xe@_HFqLlw;zYXAWSH`8(t zK=?o`?>syoEdpo2St;YI&LZJN)2oH=kd{bbe^C88Ti^EGsQQ?Xh!7VSU#QYdnLOVu z=q8R@m!Lc*x*x8q=hTm6tJ*rj-6(F!R7_&k4d}W*FxVw%zd2*^@ex4xKfs!6)hm<(9&K zpBd>R3^;sc69xU5yH8U)JJtkL)%FEm)KAtnnhS)OmkGETrMG;6F~M&*Y8USFooWIw z-q~zl#4=4&tsN;T3MoL%?e`%)?ywUGPKsaA>B99%3n^SvHG~g{rYOkYA^$ttS|5!v4DEj>f>I$ zSzpgNRCc$e-8!DvqGsIluC%t)y4UQlkG@`iX)utvs_7)JO1166Y~aBlIYB9&d}T4l z_nYNaKyCbMya%h#4r?n9=_LB0xi;Gwo%^5D5cYQ-p$7mpdO*$U#Z3D@3PUtr&qR5k z!M?99ZeaT@KZZ*>VS&R#cLm`WjV_p9wu-4!YYYv=$WZ=h8O16Nvd!V14wWF6Cbeac zf)xjhXO;R&_-p;9ckj;`WaTl$ock)gyu!)0<4EWU1&&etgw)QL$uJJDF5j-}VQJk~ z#-bQgacXCm*Pek>axDhBk|{jWrzwt3jDf6BPAwhuhm6HJd;?&cZ&5n2{e>=%&+NzI zh-FIe-sm{A!4zBtPDvKdQc(l++AGk1!YbFqlbuf6?m?^Pqs3Vv!pHK{W6=zvz*wOVl!dFW0L#)0J1U03hSZxh7|bz!y|7 zG9J!11qWRZys*8ki^mAeJUs$XX|2{nhpv&i3x6Y&k+zx z=1!l4em!YXexJ6Zsp{|Oikg1Ad9J40*`LTdWAq2ufb5%VVIa13TG~ei!3!shS_+Vf zF~15JAyulD%oJ|wY*w&6jupTj{lJmIo6b<0c%z9jTY6*Qmoqg_ypg7v{8UawqCzx~ zl-$9@v~T`dc_5^?X}$ z;jZXCkaC)J2_Bat-pv{@iVOgPu*S9}agnmurTMzyX-%z11pzb1M|EPs?J9XFZc`cZ zv?uj}Ycd)jmtI*CDE-&7eHHBBRKjM`wgz@E8B|x5Hz<9$QCEvA=BYOsl}%5O2N*30@`vJ05!z$K|2wgDK%Wg!PlvyrjMUNCq7e207@_QORUwd~B|Gt6D!@>j;6x z3ou@XUdX+Vs$-#GeBp!^!yg;$S%pAO@NMTv`8!Ay1eN)y>(=#kN!(W-7cY`Y__}yX zH&NlYCO4;_%Z^H<#vt$j_TCM1j|>U|n4LN+@<%Au`_`*9ac79V0+$0Y!B+RTc5f(n zJ+dnj0HwJk2dO*S!R7tZ@S`>@<~612G!2X$HP?W`K;5-BLDdH%hSCe`<6hC+ryEdc zWyEdaU<2OE(tz{=>qNdy8&u+?Z$hWe1vMNnS-?Z3``CQ(^TRw);J_``U0WW`*Wj-9 z6zkkcnv)2eLGTO+%RKHt!EN}WMAueQ!kF?2l|f$A<@6mLmA!|j*52sj5d5H9`$+jE z!l|>AopG%C;CUHtd%@7ePO-8*QH3ueR(A0d&$0bCF5kP;XBhBwSz9y=lTM*~Ihf=jm&mBGGkWqj0vQ26^(>un4@a4R`jcj*1phE* zPE^cG*2p=Dbyk8_x-)gVX5IhAc-Cve@Fp9clw(Y=mfqIpYyT16Dd=r}*$fdTQctZG zqq`}@n194CL;84iKd$L~BKZ&ETZ0e~3jzZOt+O#_rVJVC!47UfP7Mh00|Xw^h{puYqzV)oW0-Q`jotD zrcn~`I@qQF+rowquMSF2TxkKJbSVuj*?37$iPx3|KlB0HVW{tIE5eveuIS+UOY6jP ztRo`>Uur*B*+)kHX{{`aRDu#Z03Vw17S7G;kP#j z!>P%LA4O-_v4TtGAlPh(bUVecgydgqRVJI3(rYB@Tz84_S>NU>nYp> z8nEC{D9|J^+(cdT{)dslXuf9blJ2HKBxP^)*XjT0c7|OiE#eXHl-Mtr|6$;g^Bgt~ z>7S#6h@by$XhCsJcs89gxGbF(Q}%zCC4pUMWB>=_1Pklw=>JdsYaCfpWMj9@RA~OA z?8PmdH(-rI{r~v$g8G^;&Hp3M<>w^M3tHdR>A~Gwi2rL1gVC51kVWrfo(!n>=d3k~ z{SR@M>#ZwovUuDw z^IA0A*K~r%W`gimJ=C@Ylex69B;*E7d(?@v9jrxrc;CcV0qS=lDz0GBC@q7i;r_o< z1GVx>*<=d?waHXzO;R37@?Zc&Ha9QZ9f^8*_%-zrb@9VYH<79Igrw`DH_Iq?6%7^y9Eou-6dF%;0{59ySux)LvVN3;O-6^*Wm8%uHWQ--gEfJ?73ie zPxnmKTGdr3{|#-Xj2EzL_iOwx(ZG<(OD?y%$ANZKPKa+E zk^s-%u&eZaW&wEd<|mC|1X_&B&Gm7*Qj47QHJ_GaKyMWv7bh2>SgQCN_q%sqP&qKu zBjhv?rAiQbX5x}1b)Lr);6ugs72Kv8%eBiGb_TB)*)Ljt5ie4nUC4?)th&Ch*R_Vn zq__HG!M+D5Hm#K7bH%g6c~ZVI}N1GHJ{?y@oCl^8GsiWO)MWhd2{8vJZ4Ndk5+*MbED z>5IuWhwHw*gZ%v}tsC_|TkfUVji%nV>9+WG(gdaR9Ny0JIzENA4i?(w`_v_8H?uU0 z`q<}-zxqJFlCrY0oSdAwxw)idcvV%EB8`aE_3P6OU}yE+W?hjcF)J%e*oi>Pbvvoz zSGk#jBv?;v#_EW|-Ippufssa$a(nei{H30qE$(9!tM^k>ZdXAc!gNWQbZ_U4kv)sa zHs_D+j!v-qNlA>$cG14N?q6hnfdp}^feF<~QlaoO)PApyw70m>oybb1NF+QswQP|t zF>|d2^f$JA0#mVnx^%*7{GJQ^d>(6$a5scZ<>>=(diY00jal$C`}O&Q?uK$~EXo6#62QUDsKsHF{aubX+H#-@Il z1@Lae<=u9o#}igLc#&Y0AM3GqQ{N-_4|D+U(V6ZXhz4j!F1a#agUxI5>oEx`xqt@p zL!`}Z3RY`6&>rKXv|EZxWPx}T;7x<6;$T#^K<6Of55F*9*2@jS4o0tyLbXwj8tyP2 zYWlg-YQ??rZ;C*yTxlb^D;S8Go_~_L1#Nn|%H{WMaoKqX8u@~|9D~ZDc&qU3DVF>d z^5|)9n~U|qdqk|wWO$`}cCA-YTt%WzCN`{sIdL*I=3KmK+X$oTU#kS_z%rN8-mH}_ zA0J)a9QBH*=;-7$vLw;`r^lV>g$j-2jEvK>vwt3m2nZ2h^G_EwR!pK_czXn8NZ5Md zgauYL__|j9?x8V9EYQNLKOYjXdVTcTc+_i&-B$>&V6Lxs8a|60@;00NG8r|QKf%n2 zS-k{nIcHi_4?j~X_e((z79kX;0g3VwdVfeAPGw(K+;-N54It>ApvA-}aLcCDYMP!v z#7OT>N$k&II!P2!D@KQK0v2^02`N3L#FT?K{37C zRAity8=O8lo=Tc#h`u`9tziU-c=#rTRuH^lm@gt}1j==Vn@D#;!`w8N)vwuP$7b7v z2#St;(F3s>uky=lz1O6rG$EHVQ`3~A%)_1?qctH*iqd`3MyR0dnS0M+#`mzpW;0cOaQ0SlL+Kc5zDQN}w3xVZ{e|Ie zl5LiEQTNQ%Vd)V}Cwr7ihbvfvyOUs%sZpXYg@JUPFF}ugMFGW=#q^$r&CX}^@LlJ$ z#guA|+ho=s$06CL&0O=Ku?nSw#y7_9D!!8^o!xXXZ=jl<5*?>*RkEx7dVK!wE9=E~ z%btCsx&sC+9~G3>aj%Q}Y!6YZ>o|(c4_CPtW9hF=mTOn}Ysq(X1?QUI42Ow@^d>^T z~^4kuQ+`iE?BUVe#)*fo4G{kda-+tVZST#js*$y{Dxq8VYQsf z@owX_v1wD&y>FUX@XlVMWG5>de|#{0%}AG(8gW8=EC`5i?P|EMUD5-+-wZeR5T2tF z#+cMbS6r-A;M0VG7K%?>95+_sW(qQNnDdM*cMj(>-79!g&G|3H&x0Psq+j=6*w9xX zQPk@NV1D-YAd%{)53!pAdG0F^A@GF~q6{-ctqIHrWncV6CXY6*wh>`2{V z*gjI(BnQV!My5z~f5u=YK|oq70{%PvESAfVU8kES(n+ZuWKXFvaq#M`CB@c8V4q0~iuj$g zNbM7zw?h` z;9IrBL^unsEL9)49IBvD(1-Mg#{)Vum*=Vr<3HLCoTLlgDZ6CllM>Pr6JuQcdU zn{D=d)M| ztG{*y{o>U^fj)wBK+fl z67@#&arEqPg_%cCD+#5*xwsUbT~Q%me$1D{AZ5>tQAW&zWbY6@WA?6ra9s)jLz85_ z6sbbP=brKysi0xQi>!%R-09-Jh@pu^yQ-&4IMOz;>_mCh7;r)EIfzctmya6 zWE|9ljX&!|$lk6K>A16KSN^buCeGnE3j)s2j`myL6`)jh~Ya|{|Uf9(x z?hY1#w2^>^!`4*T=wCeuu`aVob9)wq0>LL}v4i(4hYQb2HIgrk+U+in`V7lLK6*nQ z7h~LKvv%8$@>^+L!M_PwIB_LvBjN|6-l*3aCQIZq@#{8HIjq*md197HRif$DYQ9a^ zio{g|x1(t~(#Xi2WkSy+)k=%#=oexjniW zv^ayIUviY16q+c+%$~^lVP&5E-uIx{-eNb;9q}QK=5?X+k@5ZVj@Qcr>6F{E@5pdg zm)^tpW#qeh!xwh9v4r0~8AKI|%pl$tv(0Kw?1&+IIIWqcYzB^))P{$;tI?0V?a@-3&Bl6_?EAHPQHijP*Y}6a)H=Q_?i#NZ>yC|&<4K(b z8QgCLVj)-#-#-d>Xm2>(UOwL=Z*D`MpBLn39?t#%tcscfWUTbMF^Gw?#YmKt#pyC3 z*0=>uu=*K`N5x5zIl$J>Ct2?xxUo$);NgG&h&wky&*@7SBxSUP_ea*+bcUY+-^7Le zoi3D6p!j|*D(8Tzs!?4RP~_AdP8Nx&1QGA(I%vXefBw#9H-C^Ek26(xuF2eDfpRim=9#5z0Uc-RAc}(%{+A3xE8Po;GvhJTmpS$u&{{ShAlAqX*`$L>EIsgle z>6(XKOUF2};8U+hwM{*^qe{g#kua)O(6CC+Oz+}G{nc;3O=q1n^-u7$q2`bc{xKj1 z<@X;L<~{V?n`V@OY*3U0w;$})?{>GS3OF4v{B`?Gmd&QSHGfPO|CT~%YYVz^KtlTc ze1^rna*JN!v6ihRqrIiAg$DPtR4(K&QgC>y@;N<`KI1mm^)<|~skmfGm5f-f7({Ar z*tss$HYj$wruj8K)`7`~N1QKOsm{I%o}+>9Hj&fxpp7?EExlT>^(0n&Vews0QmpMd z6J9`P%5yjEE`Kd{Zk@yBZ}z;@y$?roEQBKp--hkXmCts_RPm%%+luY_J!9J(^r6X6 z5uP+V2JT-n@d2jv)xeWo-xdh>2E{vf`Z*){O37kGD;9@-z7J1t?}>zI=|s857ZdK! zm49CjFL2IY`Pb%c+h+G8lGiP9%gHdT$3-u+IY06A5&nAFHRf*rxGB4z;9qFyT60+* z#w8U3Qh#$_izhW$i*q5|iLTFmvk@`ZRTU2k@|qF=?zW8t(uzNMhgz1Mu zIaKtyf8P15EWX;w`7XSk0^t!Hyds#+g_G%_$>Z_$HPQd)(dB-vj@Vx@ljb%{J!AHd zeeS+zn3c}?vd)!}@VOe2VA}IXegv5sTt-V*mqS6fw(4}*`P*oVEdP7#1qH)L!=Bg%(#`Zl`u)UWw!y;# zStK@!mTzM`<%Go^y_(ydw`e3JA{?P~n*Ddv-L$=mI3H~er{gyEjn}I3zgrR)Hj!UD zEiG}M{08cr*U(@3m`yfn_(H>d)^Eh?y#J!1PMG?%J|`f?T?N>gfD)yDDKoBp zo;3`u`m?&CFVLO60^3XuZ$1OB&*is9{iW_x>qJOygWiO={`*(=Ga|S|Bq;0Zdv1M*Zcx@{jmL6GXNS#r6Btj(VlW6St6p9@M%_Q~CAMxa8d zR&z{v7O!z~OA)vi1VX0$LyVws7+tEki+y%VhmKM&qACx_c^;m|yhBPGbZkN=czC^U z+gFB1=!fS(b{tSVqJ3{S4>b@|uAq!sx|$Bw?l^C(*!flE_3`iw9iFexHP^xX)#q&Yb#C9DJ=8<{l%9WxX741_ z86EBVBtRC^wr6-Z$vijg8a8}e@K$XuZt;*%TTFBu@9z~0?=IBa`0n1-vc0BF`860i z-e*VZmV}9vcH{Y}-|+rZt3Q%G#+bkB$RgO2@1Pr+3j`;C zhsm2fZtN1L7hsgia8^hs@9ytf%!1?58JMTSyc$&|1jb(p?oH^-zU3N|!ip2Rvs-d^ z>${&gz3>e)VRkF*+~K{&fjfRGL1peO0+iTY9#>xVTvDZz>!Tk_ax8z~OR3c|d@W z^o(`w2lNbIov>QwW<&PrHoe z*N%vbbK&V~<$r#}xdlRcEUMM%{ccwZRHss$Bdpi2OrdifRVii^Mdn$o@FVXoNPTnm zvA=I7Ac2%T;(#IHPd1k;4Xen{>KCw~v&QS&=7F>3fRrHUzH#n<4Pz z;-9f=4*8RWxag*X=-KQnl9hqrkNYysED?$iDHD=e)f05$8OfqFky}d2n*}arQTOs9 zF%~dDuCJUED_q|0g^sc4VEYUSiHe#HhL)7E4c3)D(xCjtv{cyttaq*eTC!)NcicSi zI)V8F+0Jt~)y<-^eu2HCbe{?fWtbJ5`NaP%XCgp`7q!TruQsQU{Ph{(O9TH!zI7qE z`+@!oPVgQ2n>eXQ_idNyj;mP^SR-kB~Zs*o#D&}d5NK16>HYN#N38F1qhKUKW(UsYe~M*w@;~#cXf8*RV#g;>E2vs%eqli(*vb zQ+PX*-5>7J7C)S?FPiLhI??hTFL1S=F48iRa`tjp8eF(fB>0iiy+kKG1-1en--JJZ1MD7C_CcIYI$1PerjaaJ|mcU{=H5tsSPv3D@5TQ{I_v)2Zx{|P9z_V z=A*Gr*(#db?&!GG$NSBFu(vB7nf@#O(N(pdMTd45aOf6y$rcU{4%A!MKp!r|<8q#{ z+7H;s0T#cp41YLxoa?)}=Z-jCxXWgLH|G7RX2)wCzal0e%Onc5$&} zeDcKgj?~9U+`crBsT!EV(AhBS0ZVtJZ+Cr%VHCOxd`#3T) zv+qmaQ2#iD!XLEMLab{+Ky(`GM1PhVUnOPZf#^x^53FU5t!zWFVCOG)8!IL)3~pm8 z%QT$kHI|4zV1qzj6J21a_F=5IK`E9A4^C46i;B`UBYjsc7sz*`7JM%?rc?Q@L`lSw zL^_J&C|pw|H>F9qV$kXJ(Si_yd^Fz>Z&lT4!G19t6;Y(7J%p|ZHCqrRgxXx#z!<*E&e#gK}DMpoBb$M-C18T6o)WRz6DS*^Z32B|e*;$z9s- ztK?3%cq;i9l!|PQk2dApWU33T!cgB z-$hI)P=k%=KPT+-@WH{ZT55`HgMM}Pb^}ivJ4u}nzMLD=jVxcY|5+m ztt=u?Kr=t0(!*Gf~e!`!Q zXEB?^*XK;i9^+t{xw{IB(8%Y-@{-E*(2yApgJ~y*$DJq=#}3n(DT1V^Za#{cdgOZG;GI? zqQR7OtIKZWWAE;k-p9D|GaBjkcZv%|B|@fpC!3iL?st?o^}mYm%Fi?v;R+u5!K4{{ z8jJ2OS0C~>A1Y3Kers0eXy+_xSx+4khO(kZLgP>EKctlr&N>N1;P;85bSxEjL_c58 zY$&*U)&fRH-tzP^Rrnvl=M}l(38O2Gcn)8t1{6;tmCM;?h)@u1ZKv)DzCG=%f9qY_ zfw|F$D#7X7?pshZv_FrJ>V@+y;D(?=;P)SHyu#G3WqcY6LFWS*(4 zMP{*0fn1u&;bR`cE3|$zn7vVFM9BG&jCrPkba;QXenE*ccC@5s_d0b|-t>#78@904 z_u7bV5{tVe5Wc|ub~?l=K+^P)WbVi z6vLbS@g~b9An-~U6;ebV6e;A6?8Tm@tET*OTM|{P&ACD%>*w6|A$UjSE4zqeM`KO) zghu7$ z%h03*rjt7}ovKsaHiMU{k7^!RYK)9$v-{)6rp_##s(yhX2hmJri&}liB!+@5f!n9ZUk!(jEZy4&^(8Td4vUq z8g&y>k1ne9k$BU1_aD%n^;_}ESpLw#(g8+7TUCiMc*wC%l0nlogqxWyQ;rxInk1dv zkjyZRTG$dFQB1}pp4JHOqK}(4DV=$yala4EXFvU-+gN?&r=g;l2>xbw`S-wg@~!c-oR1Ip%lEvdjjO7S%#(M4_=2wQR$Uwi=?|K;KO2n9?6PolDB!iR zKE18#oe(HI;g7CTYaIwAt#>E9R9F(J;hu7(K<0t8Z_9kdP|3^)nAC@BArWtnb z<8%Lc#Nis2>yN+mB?BFzdnGMFBJEE)Hm%72ieZbsTE-jo{zlS{Z`NPPX3*w3|I|w4)e7^Eo=cu~3VY)n6x^AeWm*o2`H(K(e5K zqHp6`cwTAjWwcUlb}aACY3pTwxzPvjBSn|Rw+N0?5?$Z22A6pX1e$#;PgG^K?*Cde zp)1_1Y9O9&bi}8J|D`NY6B`%|;tYGxzgQ@gH{E}GJZZ8nDnHu}MmnDPp;~HdzC@<# zaV8^p%_CAb(Q)GX{3zlS$$*|p+JJ0>RTeGV?8GQfu;9i2rz%}ad^Gs|{(Z|%-dgP{ z*_y22E1v(xS3U#XP&TTJoit&%7MH62A<6^E;|IK{VzC6a;#1s^gpfyQjfLM9 zsJe-+XM<<`hQB@Hqm3gi2u<%UX&mkCXG~e$+}waXcx!8GAZ&?MIvpkvKzIfQ2J$y0 zI$i%fR>(g~NKZTY=S+^%*&^GZDNi)yT*6zeg7%ktykKOtRLZ~Jo~+Ml7`N-d`rVOu z;u&`1)LuKB_NK94I$6)wcPhQs@c0mp*V?ojPV{r2;HbJc!PQ9qxITz=H82c_nOa7x zAn7Z7E)c?bPFWm`A{QAwoT;6&o~A$cpy6~fj+<+;g0Pv>rJ5`;Zur6Y?zXwP^zyE? zp2}8QOb*ss@4+?MIvybB6FV$3-t>Ti@ZcoelqQ6N5|k$@OQ*WwV9g&YTsMhW~d3YcAHTlv1IUsJlD-g=1K!eUD%ef#Nb?*jX2ZBHmPyM*V%gLQrK z3|N{walQHAwY8dM#iULnq?k$ZPr-BSV2_zm3*bo%axQ~8Jc!|((2Bl<484zzVS9K7 zR5M*(RcL~Wih3)xeBGJ0Q%Z=Ud?l&mrRU1X*Bul-@9uT4wdE-ASLo)3Yw8=C!;6#o zM~udE-f~8jU>MgGyMeiyVoKz?@#g7dFJ8-v(JDUr6gp|7fWX& z+#05}h`~a*t|ulYs{J})x*>3LUk!NUH7ah^Wd|Q?rpmvw+Q`p?Q!7y(9aZ~eA1X~S`nGG@nP-Y3L6dsk zhH;32A7R`Ip&qhCYb(UJUH4!Pmd+OQGaQf{^Y-K&=*tD7-Pr+c$8Q8bUZ=ChW+=Z6 z7$!~dcbB*DS}6EDyFM~rO}q>ASKNfe4-eqBdKf6F5EP8XlnD`@o}R|IC8wpC8yOjy znc)OKr+@0~>}(FzwjxNF>qig27~1$4DEL-H;IT8iaeOlYMb+Heo71CDjExbBmLk4_ zi4|^_BH-7a`R0>%rN{=6JM&j_bbq-2hlZxdPj!qhJ}mSrQ)O!<(BGy7&h0g+lvwPx z`+;EN{P~D6?t)8OKR{cQITS&NW@-89?eS63+@Ry9l}e(ONtWNGE(0djMEn!{IRB$8 z(F1I_C|@%X0SXFwGage4_E3bp)+D2Ys#3xqKRHK;BCzh`w*QGa%<1x4nTKjpe3Bb( zKqCpMUv}Ssjn05_>dTl^L#uX1BW>;OPxtdMEZ@QZgf(+SV*-hPRaSV6e;OCojoO_V z{4)^o%Ol9#yY8}CQOX+eU`u8Oi!_N?Ry=ZP;{a4g7&lfZY zXfWAk+WlWHQTqfR*GNcTQ zD+P>LSbn!Nodt_}1U+hi^`goWnbvi@ydpvVrq~z>Z*{`X9Mw%IqYaXb%T9J>8yOfF zSX~bPQ-MGqZ`V~A@5C}cKO4{8Yq%)${&$V)IW;pzcSl}IE=}#~Lp_R>ahYLKRL@I@ z+&;kiS5;BW-}<0!b@(QPa0dQgjy;7nS^{uYlG9$vBVI&5eiwDmCqW|_-x(yJa|$Q- z_L?_0b8vBRtSl|rMSp>*92kIdbG7A5aURRVr)$Yd`LcPeE*Ug~Qnd^rRYnBdEtgVI z?9aV%J$tfWl?t(>sShVwG{!%^3{YjGrJ40{IW#PjkV14RE&o-8QdZvoac2PuUOO_j z>+Q-UY?hj6^tzz7+~keeDHYViq?MkqS*goJAebPkTi7loVtu2L``_HWwp5{bteE=0 zDMNek9hG&U%+C7Xszr>~YnwSZ!#JE+8D>k(`2S~7zn4o~>ig{8{~ZZjtjr)yZQW>! zC(l?>g6rota}>^*Hk~tPbBXu{A_v-=Rj`TZk}@(Az}u!ut1b8p0h=K(D9CO|8E&dZ zLPFwkR7>yJ*Ure;c>AJ*H`aAs=7*nQ=p`brmNnwVuxB?}huJIzEp54Whx>3GKyKE6 z_u3_7!Kf4{JC+e=hQNl8jdiWs4nVmO>YZbyfg2(F#8 zv-itcQ%}O8v#srB@$v;hb+X&deuL?Bo>VdeC^4B%&0mn>$FF`sU3Y)Jc5-l_Ql;Gy z7#R4gSSB?!m6)MDoxQHE4yq=x-7-D6Zr5as=hu6qKy61tN-E$4?}ux3 zW+wg*GebkNNGKmY-V#K${(%8tcuU2i39hcLR+?-=66}Z&ZKrYsW;`ddco3*f3=MNK z{UBgS4jvD(ee4|^Y}Q(X07{EN07zB-ibmnVtq8le0szG%%MCh=0-@y9*DQn6VQ_7& z14tbW1=o|TE?rzyt;SzX!_Wkhyt{*9kRy@rczAe-8Fz0NgVsgao;qPq{hQGu1%EDz zS^(JlG7muFCDHKv%gMqU+VRvM$NO$Zyjl~@#i~u#_!s+klkPxgob6&loyEh5pQ9+3dk}z<=7$&M0t%if<}A z^sAPZ7AE-dRIbo$k+eK4R^S#jX;$BEy9X@Czg+1?lz{QvhvZ<(IfQ+ z234rElnsCamXG)?1D0>QY*c3OwBeFRfkzZgdACk~++{j2y{-S2{RM8Qu1xh$P?gKY z`lE+Df{%~S7p6Zv92_Yxz|f)l7rtHgq0BIQd3p{;;Uh!{X7PAF+Ab2j(g_DZBFi`o z$C2j>DD6ppq_0 z=`ojndW;a_4`1|6A;K32JNDCJ;#V6IjHnf39<{6Z>772iCvJ*=6ySB+6Xr7Xm8hcKO&b1hc(pQm28|SxsJtpluYG1OfC7GN;6Mv z2c$3nX54%as}7qj+1}4#dkTBL%R(f8hE@_zAK?=3)xg}mtn#!#_7`su82apujeUVs z8bO-KJIsdrE+`hTyzS{n)qlSlBD5b4KxA;T!lEaAT5NGR0F1nq+MKPBztduGlfp~L z%Bo1pWc>(x#SO9zYwe&93KAsm_!akZ-L?A`<IXU#QQI6fuC^=fNV_UKP$a$^$`GSZ%?Qy`o$fCXJF)g1r~0uY+Q z99b?_gUtx6D;LY)%WWpnYsAF7-LKf>5cUTRWELSRCs_5l1cgXmzp>7NJWE2gi@fxdg773Coe23lCfb5(U9OIRH=o z3l0uk7^xAA3NdvE7)jAIi;$P(Y>^c-#P~{!)|I6^uU`&!_p$dz6qGVKm`pSQPq#z$ zI*-Rae>d=O#piOQ>Ys12+0gNR+B>Q063uownWtxE#kjLsZ-;{q6d>8C0(@Y{_K=D$ zBikS?5w9Q>V1^A3Z@k~0d+sr$geFI=PW?6dCn==U_(K7k&bkOJCQgtHr|^TS=-wwK zn}tgJQzKxMVIP@USYVL8&ea%*CGg!Z8c?maxiB^kMdAdkQ)H$R8{I2L3DHn%2@ChH zwm59@0|A3b_q~;UAt;Qh)t`75{{F=zrXcyEs2ru#C|@U^FFF)WD24@sR^^m`et8k{ zM+Iv}N&*kr-`NS!I#dWGs;Q|tJO+Y}H>-eLQ$s)VL+4k;(-PN7p`uWG{@-z=7aaL` z@Uup3{vqui7Ede`T+gxZZ{@`Swo2DBKM0y-2?7pK zr@MvG)~Ez~$BPOJ1-z?y)L9Jw`JG~kgdZ-9gz&-*2lEj{_XlrN3b$Y~;|5Jx+xr4a6C-A@c^ zwUC{rfX{iH8lgHTI1#0UBDtDAb;yBvZXi+kiI9rMk*BMg%+Zm8mc%^+HpOt~t9Gr#9&LmUL0Ao$mJ?`U;Q89hhr}y3ToPB`=BO~2NvzUnRnMgQ+wEL? zC1O1^&pPD9n)Y+-tiAi0E@K{v2n=Fl00a=r?X}e#1O(Wh=@qro_$6WuAl#XLRxZ-0 zc+_6AT5I7Uccnkmq6v3&%4Xc!QLq2H9i`$x!=Wf1=$cg3?k4_silJpOw~Yx? zD(&qK*9N~U$f4~|hUL?)^p%!*!&eV{I(*fpe=MtcQ6{edQTF1>)q@Paz!El~N+7FojBY9`5 zm5=exP_G=o7y$4cHsL<&=Mz$3TvR>luf1Pc;W{&_n4t0Tf9c^q$Q6#N8X5~F^7&jF z?8EkYE!9*P)YmsRdt#bS2u^%G>aQl8-@|Q^YbbshRni1q5*Q}{2$!5!Hz_Fz3m<=c z!=GAKjGT-N0Eb^Rkvte>hW__A@N4Vp&>Hv?;L9;La`dIZrOJdv2;KjFOvz4%pdlI> zhks4v%`Tq*(N1S-V3e7fNzNCdi091ShzV~6#6$si;s+1U6aJHt=M({znhUrWvVPc-+){{S#lQ>tCg(SbH3rniSn*^(PDMI`=VoS@5oG#pwdj6A`Ka* zoUy2=C}2u;ea6b(5|^+7AYUM>91RWauWmRUb`nCwwKfa`CnqO|mIQL$XHVtlE%JS5 zqo^>_rucYgbEk8m5*#J&82!rzI>Zhp@R)4zipKa_O;rXuNp$-Hql{5qe`*~vVft~!cB1&6Sg?S--aJ;g zl;kX}ubMHX5Bop=abTuMJaP&X`|0p=bXa?#B;U*H$D2JRqy}}bn|*1=HpD7ah34kw zf`S4(9`||x7XwrXc%b2P*y;lCm1fyqS$Q6q=76dEOtqfS`9X*#1IN*Z&&Th0cQ-ey z<@!<}%n%4aro(Pbeu0Jk#>1oaumxseak9F)8W;#^O3nTm9lb`r z?Sz#r*{^&WkW$3OGG-_JuW=l0CGdN=3kclC>T2IjVzR|_O}wDbrhPz)(%ka<29Tfx zjCE6!%TwYJ;EoKexjh-sI8nO0qynb) z05Ohrrv5sTpPw(A$rWT}Vq!uCSvLG2$q#pl{`V4HGCni&cfK5zMq_QQAb!83DcCf_ zD%dSFrlr-utY2v86*dx$Jj*SL((Ih@qhzj~8A_Uhn8 zAtewo2qn;`_Zk%x73o7zR%D4O6Y92z6HN>dANnNhP|f){PSFYbS`R_@L!#+cMJ5o} zj*b0XLLRXbE#T#VTJHIqc5ARaNQ!+FY8UE>A4d06MkZKfQ&mMpW#A7HMn4XfFk1@r zL_|Cn7uSe!AzRObq#kv9lz|;W3?ohVT|V+esPpa-kxqR8(lC-KOj8k__2=$0NGPaW zS~5h-2I0OSTXRSDed<`4@7H@CD@s5ii01AARAfMqzE zL|C{a77FhvMGg~~^lH7a<800@HRRpi6!AQ8khml-8 zJw3pMRLIE<+E2$L_};smEeAQf$R1j_9Z6az0fBFj8Z3S==aqou>ad_D?x*oY$F50$g6nlSDe-530dzb|2uQ z!=gc>E%?6wmwP4#-$n$M&$<_|b89mB?O`vc!ET3jdlelgmkv+Z3m` zPitq{mJ9Z4W7vx4QDJfp3IOMZ?g6SNP5WIWq?_}w^V_EkBqL-*#Dr~rS-~24hKsLn z(c0kSqr4$?y*jT6RlfudR1fL$gu`5?ax#+hYBv)q(*~>bSeQTnjM`-Er>^tZyAl(x z0cc;0X}NMmHzAM-#O;1drMXx)$s3^sgN)}}3IbuUe~SObDluW+odpcFJMf+i4Gje# zim<4tpi`wzS5(aGDxSHpj<0aXyaq|HVk zb-6Dt1PM1-6*pv4ax)n%X*-58%XRzsZ+W#<(&!op$_|17g5jd&AWMkSW}_2-0&UNB z$rS4yZG8zyNL2_};P!syU_kM)Q*ias95bR%7)ESNrb$3ZGhb<>zs0kEx|!t5&&vY} zFu^&ftHaEE;i z+)A@ipvxhcI2bZajsMT"xHrR|F)YCEzzO~tVr=Z8UeJrz04ClG0hXwj}Db5u4n zSqM96tDimpAb|E@w~S<<1BSLLfDMOpC9&3Zj}i0T5=`vX$Ruh7`b=ci(}}byQ*@DH zDmOW@u;OUQr1uGQL2?+aP^1Zc=}hu9Q4)oMb*AXn*j6M4zBp3)LcJj;0)dzW1Psy` zGGIxQ_NDcjo3K;a;#GRbhuZbg)*)7XX4|K>{V~YRsMx*+TQ8g7L}-vNk743!I1&s; z8vS(1Bg0VPNK$412u_P*2gk{65~$JE)~x;_B%(Po#g2(gk6L6}ge(g8f zi3EY*tM9@MaZx}Myo83_CAz|0R7eE~7u-`0*!eb7%=?2`5zS&Uk!mV!+3mVQoS zf>ug@HULjlbSN(^(sBy(8Vt-*yi~6WZ%P>?%WUMfG_gqv1Lz^z5-;A(_<_l@k1`vVs0T5VkArBHfL$yZ?dv) z8Q-U73*W4+Cg8S}fhQ4TAbE}I#(g$ICjR9Jgks7q?}2b27I9MtMe zxnRLR1uPN&{@BXlVU^VLP1QxBV^9NbHTNpQXUz<3N~oLE3@STDaqYxZ{TJ;fyvwx3@Mmyh(gr91;Taisp!>278K1 z-u!7bUsILXBvl2-Z|fDh^HR=qlbO*2qA)NTxGLe&T>%^>9pb*rUXBcP<)JcmvxgNVZnnb7F@ zWY0<2<$9&>illl}YcvWD0pW3b*kezv#b654GH}uL5p@cNd)Qa0)kZTuvT!PiwrOsJ z=!Zx_K_M!ML=6tVcLw^4zSu2GhJ=7c$N%ecI!1(_|5K$iP!_Z0LB&~-dfM|ZfKLU~ z$0opk7VrSdkz$oVDmyaYIsmmo`4|3>5f9%YI}?jtP9e+u_WevTUnqrLjZxq$C`=Ee zp_Gtjm&QDZiy-vdrspW_`sj*_!jh?tvMnj7f0^VpD z6)kpT><16&+T=#p@*|%PLOgCFgUFzqMM>#rZxV)Cq!M{psmq6lkb)X8rC`UsYtw!< zBDlbKxrh3C)<@n&GI+fXpsGD`iHu1v;g6<|jcl`2TbO~%&l!onQmwESDk}rplC)K@ z+(A`kWcVDi(_Vs77c$=MrOT)xftXTSepl4 ziSA)%)f{`x0YKR1>&fm>TUi1qgr47^0d*tfL!4sG&6g?v*Cgt8jeODHfNllOp}QeB zyCzrjGXn&rzp0!wFK+~`aMQvZIGvt(A|=K;Uz?>aH{_LPAbn zm>2DhTcIhZ#{1%Tax`CoO9#I23Ghqyvp$_IMv*C43~eWv>uJD?mZiMfgw$36 zk$64gQI1inTJN^)#mqtS(c zRzSgDi%59;CFxyX$HJ2yCBKIFYc_msSsP!()oOJrZ#0$JMa@V?OABr&bk_zbnh=fZ^#v;xQ;~1<_xBU&Q+aL5qf|>y(QL0V zi_y{6Ea(O?Tdr#=sYA)(qwu9s<-h~{o5yzZ(@!bU5yT{fo%azaYAK10PzHRUFn`^^ zyjDhg#U!%&ZLjE06|J2nELsiw3VOWT)04S0FM3uOxCBZb2talu3tKk)b(&uufJKRA zrnaXI6_y~wE62Ej+((suc8lIt3yLnSgeM(3*7<5(9AxH%q5tRjlnNkHg{vVC)CIb4j*T3^0_?l*= zP8G4rUNvJf+`Qk<7{S7d&%d}(UaL2l1HB<>-kcGg0Xp)GWSiK_swAlH|B=FVWA*)g zpvQBaD=L!E>O+1jiKt9F37z;3w0wIAGqC-()N{4IwmRk8Me-tq&| z2lFP(aTRr2sx?&;9x&M1*#X(kb&YsO_W)5xVsU#x@62Ll0uJ}nG~s)W1tvJo8^p~| zsL`Mvd%XlcHAb_>+O#Zwsy#EZ#PARX9EwkCD#&S=5D(GcU_-h6MAR~=tjOwap;8rD z%~V>*>s%mW8a)MQ0_0n>#H~kx=I6*rZ*W5c+uR+Pwv(~?%<$-_fsi*7k&dx5BS&ps zk-DT;#Pw{tW}!YJ4{aAx&(}dJzLmm)0w7oENUz6j%y}>db#ZB_hB)2&YcaPoz{VYb z!X3?!f2@LXrd;W)&0fZEKG5PLn@mg{j&&YnsM>s={9CG6X5ZNv-4jy}=qhM7@b*tn zHQ(~l5C&SdJHL*P{yd=5FagvjpbzPSQTBnmhNt(OB$rR8|jkn z?(S}oE|Kn%5RjB^1SAD1>5}f|EZ*<%%Y8Wl?%DIiT}$aI29rNPgXVS1{t$7r^(p6= zZ>-mUXUBXh_c}cD@e=dD{X6?;YaSQ*cylQE^~avAZFx{nWsx8qFn4jV?`r&ySSM~` zc*1~mwcvbR**^<0UbZO6p2+#gHr~cFcrFQcgCU*8u=8+l?{iibzezv*F*>-gMa!;? zQHzdNOg7S*T0=p0(X}be4deo-{`pb11kc2H8B$8=&xQdp>|wh!b#Z<5}Dpm&uN) z($dnTfWd(8T#W;%cnEm zodmwbGQ0qo7qsliWM0=Dzw?oII8Uo-&Z5<&zl1m+EOgxSJbzjcK0{ZyTF*JocCoCq zNxpn7N>V~V)VpNA+Lx`UC?U?xlugRm?}cge%dEehaYinZ^6}x*BM?{s!2jmv<{T?u zb)>AK;_s(ZV=1YKb1VZ|l~{mn%k6mh)S@u>bJAe)*X78WGaHgzqf_v+|D0Q?1_bbc zr=)~LaapNGIVZ5l7P}4a+RWFWfYXzAi(d5g^yK6c>O4zuK}ttC(c8BVJ6VoGVE5DT zO#oxcJ$M!HbTv=NX%nO-e7hD(r&j8=Hwh7Qd)$7~dAkuyPEPJJjYp4ko}!MzBq0mN z6;)N$4-eHY@SM{#=)W~?y#4(v0t4$C3onslVyQoW^uQe$z!HnZWkT0aPSc1U$`ugJ zdr44rND8GaDORo4xm=f?Z6xs3h;ZHj5vtmuFB*;I~!a(I~6P z52Zi}M_JeC?d`2Y`3uRtx4yo9d~EP+Och%{sE$0D`{8gcg>+}WVqtMHqy#x#EA@!X zf^`&2Nni>rUjXfCtaIB}1?)6yUJuK+5ywBDh&IYR2hjXHYq6yY;ECk#=bAlPdt@?paM0rz6Xf$LrT7IJE9J1OsIlRBSUIU zRQ?Yfqy-qvr3Op1h2!ZGc^)uQ1p(N+Cc823gL(u6Ro~t1tq8_8*Mc)R1hG8S)H$n^ z$`pxVbS(erVQ&Y;WS=NQi-@Hpgsxq-h2$7w?mU=WjUDqU`X zE?VJ~Lqg*A{)3$%kV18-Stj8fFDvGLuW-%i#Z6AJ`fHH1)8rt&Dl|w*V6_Q=l7dxM z+!mP@0c;SW0Q=3t%8K?9chuTs^jaB`xdvbvz}4OBp`)`1=wi@dm%!Y*K^dDood~nI zw8U*QEAzX{WjiJ%1#7q#(#|q*+AKorc3zn4xl zs0SeEdLxqL({>@ZGFpO(`8^&%Itf7B!R{KdW-*b?3%ujBLnA5gz-PP7VTDgZb#o{I zPI7x|%kEG8_GW@`g!E_Oz~^LY`MO|Snh&4fOr(tW8U{5c&IKP$YoLoVTBbPJ0%&?r+PxMiNshs zs21)s46UZk_yU7---=rAZ;_Iz4$d2tl}SyevJ6yvdt*T655dMtB@)xD6h<3P4qNWa zQN z_)o}kP5Iv6Ltw|TL58ip{f4x0HH{_;lL^}`-ix7O^=2en&TCBM#kR(?{>vlXi;j-Y zw*`6>C$z74$>JtJaseZHeK3b4oYe4BMKn~v(%k&J&y~%>BRGtf$8sWjh{X59aR}y* zJ5aiQoJBB0tz*-|^#y06!!ZHAtCh^k4hUVy>Tnn|KV)&CsLz5{llm{dwvdXRo*oST zTwJ*JVw^q3rz|oLPT^KfR2n~j{mRgZl|lJKKG94m`X6dc ze8Jtt@N`IWa=F4=oo~r}ZkTjQhn99cJct2pUP+|^vYV@g!t+3_zqh3A_DnSXMO3J> z(HpM%Df0O?w6F+%>R&fnu8u*s_VMIb`^ps!>6g$+(s;X9}_&wkr9lF_1u+^ z=4U|*dlncNoSq$_;9hCV0jY+# z^hw2EGSzhG-S_2OcV{lO8+VkFI-VMmB7|jyCrZ9D)*;@(OfFtEnl@V*_xYG6B3#-4 zsRXC(Z;8(^K0hZWBFbJ_qfZ3vB;H#|CGr_U4~Gq`L;>s(vTp(qn9V^6ghL})t}}+} z#m;O_gXX=3u65a;E}5NGm*7gr@lMHrAxa^Nrhm5Lwf%7QNskgL6dOQk+-AdxpaN4_ zf=as!;772jp3SN$rI}p-fK*B7j*16a1=!_h%T zq-QLMfp^<@RN=2{D6bXTG65iKBg833;hC8C_oj-R7@nV>hw+?cCuOYO)eCm0VUyy& z!Pxlm-DVEbUJ396r)xb87ULl)Yz1{#E?>+kI zIcOq2ajM@6q@7liMlQ|;T7#;;49rZK*t%}L?$-oWn8<^BnPY-`+yiH(C^t?unP7H# zd86%+!=O8%LvcyTyRXQO)U>0d{QUN$ke^P}2n9ZCex-nS`^B6Tz{?ZWf1KlKeTm-| z5JyBjyc)~_v!UAaTRcLUSon-|pHpXMD;5GEfc+ng6OK0yi*f-*kAl6Kd7$^c2|=sg zs^B9Yr6#o1Ckn`siHNwM1%bw(hZNH*_<{T-PMg`ig>OYf9qJ|G>ta3>j)X@Qn+@pJ zwKmFD%?7Y;>9@JE#_K%ICNgB+L`)1yW^st(|0pk4bzh+Pik zCue~1g#)O%PUwW`SbLxlC;3Ma&U{5i2ZJ_Q zn1|NW(~}1`Y%RY7v3l(5f^9xQo1a>1htHcn(MEJ*H@8a?CJ~o40S-v*^vg7a!!+^m z1$Z0i$3@HdX9<|lY>ILgH5B4HP4TUGGyhyOs!hVlCzy5F7xf_jXXBCQF{`09Q~?gxu; z2XlB(#hm(97c>V;8DBUGwy;&S--;&|3XDb{%W(Hdc{i>}C!q#B2v+MTSkpE7XI&D7 zL}UCFu}oA5QRoRAN!Z;w-OjYUZF0biCA&^Eo;XFv-AKu>YA~!K$%EB{gq{t z8p*zc7-9WfbUZ_Oe)W%vjrL2pcRckjRNlL5s^3!#&{@qA^slG;4jzozvwZ-&GrfpL zBCycu{~#4d2-Gk7-t>|+F|XrBQu?d*gN|;br^k4q&X_E>dDRCAa)3bk6aG4AUSd8M zO07HtDoE2px5<_M*YY)xY1C`_75TJ1?i7@O zub>g29L9%UuGC;q!*mTGseV*&(on_JZ<8SPqs52eroAAywe>0P`gl`5QBuc>*hNb- z_oX=>0&1`C#`+)b{VwUjoIE->?`bBbZ+}O@#RDT@=IM!>6B(mQ@+(t?EG;>u^{U=i zbOdB7`*$(nTI?qPou+ z%(u_%kC4>;mOH!RMd)wCBO=6ILrM7Q5+rLC7=cABYD*cO zGXMxHJHbxyDjX8<#ncoAZ&YPnptp*DP8SRPaplG0< z46TDbhACjjelu!>qQLx8z6GXxdV=XS!xa>So^qJ*aHlWyY2FqVnn#Pdb2I!r)}6Sp z*7=LdeUgk06ClTXNds5^4G9|79MZP)v=?bjAQ9KCL^2katQdTguKR;3 zuw7V@B1cxis07C(uJ5}uHY%vnErywZD$U`zN>r8@PzDG-9?FT>Ml=LcHpy~2MXJd8 zr>7_9L9lO9E=z5{Jnjn5{qkTV3bq1s=cMEw$}-GGe?45gazO;r)d5{;z0GWX#*k_x zYVo#TQrO&Ax;?n%2)-zhNMXqY)ciq(;guXd4{`~pwaU(?y_Z$&qAw7^vlX#x^yo{j zMg57T;U)MMhs%<6IVIG0_fc>N-OM}{)j3`~11l$p@EK$Td{$VlI#SxnPn4DO9StyxEu`> zlu(3jZdvn3s6&D9~PVbpus48qGl5=lt75#M**wpgD!<$h#u$8lF!I?i-xZSCPP>wWC-;oP_f#PbMF zZGvh&aB$lcKB5(^Ve|iL`~HQRX+r^JL0Y7a|6y7UIn0+NVr$Kk?BMk#gc-lSkJ@J?p|JcL z12%)q5rCK>tC{onA^gn&|Lk`qcnS1QY<6W^u0EUT%|B24x&?$;^04?Lrg}dhNk(of zT~Ii^00Ych`mSz?o2$t=M5Maux8dMRI$*IQBen_7j-wpKzk<9Ma70M@)F5Ow|*kn(on zAMpU37^q6;;z?|Mo6IJxmE)Lk1=>iVnOHavmkAKP?SYh;#TRt2utgu5LYsz_uy*m* zNwmpAd&(=9za7j_2__fuYgOPQ2QyqcyS?57)?@xL`Ozb<^8Y!1q)aS7AGvj{>CmX- z(z7G&Nd9-DWE_C@i=F4ke{X0+MSV^WaxKruzL3g9t=cQnDV4TnP)p2^baZKeTy6KZ z?|&{FBXx6!GQ)B8G2AKc~|^|JPx*h@YDwGj1d5kO)``UwgTnhwP zSd;>06*@H=KNydkbkhk>*~xy=oq&xsOh!;?(H8C0HV6p*&PmW_;p!eg*sEIf6ome4 z!^l{^(QJ{P#}Z~~u2Ep>%y(%@5cxUNcFf&C1r5bJKlEKi+j$k9s-wHGlgxI2K5WsK zl2Qi3js$wz+O}%?vV!}q6eYK;nH6PbvC4QUBzD@gC>zOwAHNR|?eOq24)%#Nd3&P& z#Xo4Uizwz(yVorC&Ghw5^-PVZ+?{M4rlw`4rRM^nZH`N4GoioN@p5Zz*Yo}9df)jm zP@Mv9CW;va85ue!mBN}RD?6JA7grNBUcrqX=2XA40X)E(*w2)Xhaq6067fs|kx6n5 z;`#W;au_}}rX6f^VCEAit z6`7e=%cSZI>DwrTcACM>*|zEw$ZSQ)83{NabwWa`DOt2744PV;`zVB2{*EB{*Ptz1Zp@vWoD6I{{bf89v;E|CR-awxw*IS5aGR-<4UwlvJG|O|YHDhNuMTkk zy7O+RhIe*=K>Tu*K48iKT^{JZhIc2cZvSm>Ygi~ld57YGA!r7G%(u6wRzfBq{gU8yY3*-#ot&w*LCn z2orZfQNC#Gs{kC_T$E+GsQC1Cb?M%FF0epX0Ja=4pQ_py=ikcsm4;Mx!}&k!Jb_dPzAJ&!lX-eGwX^nG|5L+aY&Y60H&wU@1*hZFkbk7(P)=ozVdK(&zdywoCwTKaLcXvmw z?0fia2zVH}fGY`jMlfrzdb9aGkHKpBO7!KPVM<+v6*zGM7(ZAd0-1oq8a~+9&+ieq z6i#n|M(Gr7jhE*=&(h?ivV3)VuY31CP(Z=}iw}&(u;bpZrIDj^vnMq2kriXdCa%P= zp7R)sD)e_-i$Jsu`1fEJqmiE(EYcECf+C2zH0uRI&%pXaT%Zhy+e!6CT(BRg!MdU! z$z6$U1NwjmGEo`ez@h?(Dm~6*)c#jWc zu>rbqAuEGORAS|*DwsQR&Sk(yfbZE3#1O8*Y|eby44^kSz@u!46n{n`{4%WI zJkL|)jFBs)_D1LYM*K7yzsKT5sB$~=Xg9KW)a50>#q(#4Zo=5OQ9dE`=YZ!eJ#jl- z?LS06CF-u7hI~#{Y0k<)0f8$wzW6#df)DB z+FoAO=->WXH6YMM$e zdn((v=NC{+RiWZviwbG4X&izla!a`EUBK(-kD=%hd>#|X$m%%OQAu911Y1A@+WqHG zD<~}B3xx6V``w;aYE(TwJpt*Suy= z(z!R$51TL4U5FAfr(zGjz%e0Z$irGB`O(IGPJ%UOk5nGm42PcG zoBaz{-0xaSL((s2aVws#Y`&Kg$zl)JDs}?;38gTucDu}Dc5|c0rx-vwK8Zt3qfH_F za9|6`>(8!mIun>%))y(!)@1+I{lxfaLyWXj39oNpZsLB>VfFEl>xeP^@xw?ouPyh- zJR$R^{UI|;`CsvcnGO3L&D9B{P$?BccZr&{rPsu?8x)+V8vZweM~kwC{<5MyZtFz6 zydS??Nv>mHyT*OtY|cEZUYtp3Bs4)*hf9l~7aPx4hes*s0~#7&DIsqJoWompA|fIq zHRa;6Tpz?|l?l`!jO}GOwVVpGfKzh#T3f5&ppS!3=;mT#kb2a9FxM2$waN`U1JW$5 zwN+~6!->F^7t8Sta6)X{ftOR##^1o^3WS+BI0*OXM<%VtCQ;HRmc)i1#pRbnBA}hXbO*kFE@et zO2s|^2;ydMT4!sKM3TQ%wC{D0q&LPh@fcT?OxJA2GsVsC|HAmagP26BFS6?8Sz7sL zQfQ@ohD1TtTZN}eqfzX|TNZmgrJsQeAD{5tk3{qQO?;o)WXgX0*v}&PGQKoClhv7H zj)f?fWs;Fp)1BOtiD|hu}l5Yc3kUfNm7@j)sg@vCrF+_ZCmad8^%j&0vPOA>PqmcWF z=U^iyiUTtZ4J=nZSaJ4PIp8ofz~-JW zsZ8D?hAq}tkm7%T34tUARbX8n;S@t0kI`%Nh(SbzjMkuq)IS3hDDbh{tRI0+8_+oD zoyEYHLO{R8O@I0$CwV=y>N7?a?A|o+<#s!K0}M4%s00PfjxGS0K}NB02`s5(B3;*N z44QWNTgZRZf%VYV1oeZ!Y6mQcp~~)f55v)db+wP6C}?P-PTNcsN`lsr9|$zRFx(*E zar01So{+;T2+^n(m{JSSFQfkj0UiG8UNXX)vFI=^%>!|{jQ zAKsA2cw(FijWWeQEA4%cjJ9nt?0oG}Y1y6b!zDCvfw2+DB`h=*PmlJs1_jGR?iaY} z44R2~wf7e+j4%3VY*M%4TumF5HBu@@E7|cm9KU~aDP*ZEcpsfB7Ip{OO_DSB2`B0_ zjiJIzFe;#47w6n?lKg_FKA9?l>#YUkI++DU&i5hZ0yN=Ablx7o)tB&zn)bLVH-Mog z6w-Tof`{^XW66aB9+jSp>lz5KWj264ckQb(q;P~X$r8GcqM~B>?@IMGHIMU5Xk4Wv zGy0DPAy>H|)uV<5vb}$3NIU?{dKs~>-yqIuS9jOaK(Ci~P-{_ETo_$<3?(Gvv<5M^ zq<*Im!=(TgA64&KMluXK$20I+|2}W<@ObenDJIxK1-@0hd24-?&#~1^D_5sL^-=OV zqUG)7HIVR=Z=`d|^3z4>rr08yg@R9Ih}I$~fr`z5lkPB^T`0sH#fYzeLtItQO6$h{ zaVYCW8VL^o>ECUrF@%3y}=akZ*wKG8hj&Qk>*74;d*eL|i{=O#WuD}=ICj@jZWd%|@nXh0oWX>DWS2Z_VUD`i|MAmim&uxA#YhV_nmZX=V z?vVn|*7@04aM>i18x`+J8Qv=>SbEw3#1X(l5*HsYW!^u#VC(3ZP$eb==aWwP&E*?1 zMLnRN0M`@<{MLX>x~=fswt_&6!F4(W4)PU1RRb>xXzBmJFu4c4>F>eLlHqaEG(6EH z()pG^7B&RoJ(a@lgz8Ih>k9#sW?;~d@v#yi_;`pfVbfETO-UI^(UVD(CTzb9_&@k; z$6Q_EYe>Q7@F;Tq4yhs{8Xjlz?4RMaqUYh^uB$QPm#m<&E-8&l$~2d^{*8R~cJusM zPh6-oWMO({V6WV02vW6b+|`wq3m4DMpFu+}x9lA+Ke={eG(0!lgOqeitDLDv){)t% zJPhGQFT;HIp;|)&iynMvMp9KC8J(5N#M8$%3hCzpCg9K}d2b>CM4#LFf{f;1egRa& zJ+N06EWpF5U^TG@Jl+Cj{inCwip(}fP3%oyQ5B&ASim$YM9##7A~gQ{H}1A{V`6Jd zc8xf>SRn1INCT9sEkqCA&)P7QnI*zT0ZZY6ehlnW4*-RGuQHXnPhb{c8K#>rt0;F} zzO53_4y5w-pyVc21@|S6(e*_er2?%DPM{nyllU}n?*LjCmXRP`*yqwJ5hYb*A~D!; zGL@(h`ulJr^07$0KA5P$hy#6N{ul}`NQ(q{tE8bM1K}`JdZ2ol!}C3@{^Qw zVIn6Zu1pRTzaouO&^ACTrl!$}Q_0CiY)t?uyYuqfSX?F`Y#}L!QH?kr&lP-I|MOSv zF&=%-TQ0C_f~`bZ|CGQDpCjVb0X$ZKz znhPBNlnh&v#cAsS^!Q#>!_4pBV=T@AXW}6uDry84V01}0ym+WvL;*qpOuLV%SYB2T z$>FF*EJX6WPt}>2MRW!l4tvjYPu}fRT_T<8)J#lsL-Wl(_cAq2ZwS^jPfTSXCaO3;_@pRV@IS$N|+FqUH9N8@8z%g(5c6?IMOLQhXCW1PIUIQEdU;43Lgg z^FLsLlx&#=_Tb{js6Y(TU`b`TGLfuT)z2TGXs2Xmt^sCh9-=|CzrkvzEEFcExEnOU z#&p@>r`H^e+oBCUq0HFw3Al9fg+8Lk<#R$m$V_$rIq!9nLTA4Ir#Z3(*-l-3eHL%d zMD~b59lC7t_`N$ehT_#hN)c9Tt0f%VlHH<=`@c$mKPJx;3gMT-e6qJ@0lHjBh<1PE z67~8bC_37)lUAzR{fuMm)v=%2b5&I&-}8Z2qg>p!oyyQF>L7&)HG8j~@# zut4IID2#x9&!N837PlRal)rHJyLy{JU(vl6~lJW>n$Y;=|~G5XYu>by=v&#-5{sD(G|HwNA|x zSdlU-IKJ4aM`y}c<_&iVr=7AFZG|Z^;%8?+n_V14xxC^y)x17D8JPVt(MZ9po|aNH zrRY%M;);~V-E|@AnE5MuaBMuya?qU=6R+>n-JMV5u-F+1TaH8jT&R!AxSr5nwSTYv z4VxPpo}2qNqTn6R)n3j+*ROLTV*6Uj%z<}i%~s@v``JX31>>nKBYzDU=d%Se7n{|T zvWq>OQjcOS?CrLWqTZo0F%9S>{18YhDe=oyMU+1nd6G*`ou{M~5~V?eeHV7wm}mSpOLDzM`C`8`w6Dy zj8{17`)$z>ylRiq&h){GX;tg7tIczrmh#szka53?U6WaQ`w9@Hqb5-Jw^2jX6_d|$~yGl9poK5QY9g2+paliN)AJ;(NV?cl$ne3w6R1;nVATve&jP4wi4{@Y2h@wUMs`iIhf| zT($+qC}vWZsMG%PDDg~FRsFWCc-Q~$Nuh47>C^GgJmexO zf$*a(kXZ43&DjgKsIq*5>@3y5s_u(}Y-Z$M0IR5{?#)|~wz!{sxHt+spCIVdM*I3g z=i*fY7UkvSlK+Z0d3Z=g4EgsOsi!%wD1@tCYu+^ zN0Qdf)kPBA;{~rW7`j!m+aN&knjVLYWznvBTg!vNw*KG$dF#)FQEr)P#pkX6I3fDF zmaNds)BpZ#(wbv{swGguV9lEH|LI>T#{3fpuYcKNr-gXs%>63X-W0XuXD|IPwG5xz zp;nb{A8Cg=nF!6|zrPIrb>vX1Vf_2Qn^v*qp&{b?sl6KvqCkCka(I}Xl5%nXuh#Wo z4E%0$JswieCVLODpkL((In#3rDmYH4iW9zr8=@H(du?aWzc~mM%*?jHQF{Q=Q({{)>8m8G_EW`qE@5$+iUPV+>MO9RI0stwvJ=@I5$pM0Cu+3zE zbJW46({py*@nipePK3u%{=8vP+tu($ux52}>GccJkm|$u2Y*eLa%q^zdPgH&xpe4UiQ&k1%T}EQ!8aPPLY$O@nrB|^7 zE-r3AE1_PepEoeEoCcig|M~L={tw1RXkR?3Flq((9c4zCMFN0f7?|OK2--dOE2_QS_v6O} zpoIXpi3A_NyfGV;LQE6{giKD`^sje7)CY)o^WjAD$p&yyA-X{k80Fch2gU)a0w7J? zaG-40s+zOHZSy+c8Ua~UdYFn-x~pIq0_rmzTZr!wqZ=Pt8L){x7r~SSXf@!h0$TlM z5Df-k_#4CSySvMwli45COn2ZkjO@amP>I|dnZVJJg07dsEa%x8K|0yTV|}*`9`E0U z&rm>O2QcJiJvsT3&7pE_*Mmic**lxc{I1v% zoLwwy%fz!a>@5@6OeXqVVu$dcUAzAkW@nL2Q@H| z%$JdjDQhlz#FrQex4j+B;O0?^%S4Xs^UQniM-dlLP#50yT>MLjJPkzamHGP?^t;_* zQd6nk9B*w?-;=ZCMF=V45ae~0@qj!=KzRUA3C6TSyP>qUmId+V@Q*8)v*U^Rd!>6k zUMJV>M-U(c-j_91Ab%ym!U_#a_Rlzb6YL9CFW``lvTuR8W#&l%?`dV@s1guLsxWp#%bJ33-ZKJ)EcPo{)=#&0vA1XCgmc(b+)pTh)cYn zA>-iSz#szKMd4#KU-IJ`4G@yDhmG)HV0IF)TZBbK)abRIgJ%sa)SVA@9nAaZ3Uj8pYwFFtQ9)9 z??!(#5RpP3y@gqd9v+B~Cn{mLEO$1w!IZneQ|)LA=@!uec$6XWy)l=+VCcQD+~4t1 zYqfG~S3cL6c+xT@VZ6idmw1&Lx{1FtQmoEuXUkZKg+#>j(AD8u+NS4cBun9sADx+> zP6dGm_|M-MzC49u{r0_|(X)k|b~=r`_Iz=E>FJE69qkpOtdi1G+=esL+aV5GHTiUg zh$JzwWiyt0D>v&`)(2Fh4EfJbF!QLsF{@soCMH%}?3d3EAxEVjO6d%z*eUmS!yCOR z?|v63y}((D{|$V%a6E5lwS;R!Bxj7bH$SkoPG|LSD$3^*W-(4fIUI*B-vKxRWoufJ z!e=c5@9mLyd0|Bor*PlHcKVoso;|1z0r?#zI|CDPMA@eYzMw)CsbaDmRW?Un%dk=S z?Vi|jJ4eTplM}-N7Iv*3yg;w_1Nta2J3Ax-Ug(Xp?VgTxsCF(u)Hb{=%DPsaJUz<& z*AGjZ%%*qN^Kc;;8-?i+gtZNK2LL`q843RkxK1N1f=n$y^Ko1ipo$;PF8;9w-|a^W zjYBM=qYQ1PWhFhu3)Bt33K0BDq|XY`?;0TBWHy6Rii)W8lzE}C5RnBmn4+f~*x!Jn zl#Gd$NC4v)5`m~@x8r8@VZSUd*Lx=u=#n?mLG?`Z-%$0vK1AS3!)pS#1HdWBme>({ z^0A~6Fzha8Sqaba$De|IF7x>UE>B=}IzI*I$OL{Z$nCJcQ%$-Ff_$e`gu`Hc&nN~< zL>g&8vrEWfO7rwR-RDXSl9{J(Fg-ilA+PW+C55E89Jej3|7n3yEkT3TfD4~HW+}@= z_@(W(&B1)$vp&m#uC}U`+Lx!CVN^leSP3j_m_o_l-*+6vKGi>Jy~l}EYo9Itt2xy# zJ6p+Ft;2l&L+#!0hzeEdP`p^p<(68b2?3Lyh_j1BLIAq!R7ADb>@IB`I$=pH zB+u#rB=`_uP0`cEvg?e)%E7Qf%xRO_Gh76gU?A{khlfRm$U0eSK8Yj^fJc(P4==}M zB6Ack-9A1>#e_NPS-Uy;o6wIM8CtF!^#iddDIp=j;J3^`Ix%@AkQ1u{JP%-nrw!oy zzN%U&$t=9kc{QWEcn}|VH#Q&-+&w%rE7YSpRX&45g1>x;kwCvAl|gmy-mpTP14Yk? zvBm~q5DPFUBs@;}7%*$1qK)^S@9Uc_;)Z>>>@k;9zR*7_J;)(r4(ee=t4MuWlid1v z(xEUlwY3&RQv3*M^H&sE4Dsz19sx;++T;9@uC|L{gx<4JjA zz9-s)m`|T1VB83KM&`BZbR1ACE}*r@BY#d5_M5$blz{;@3MQQ>agw)U{7owT)Qg=8 zthP^N_ZFamj(xcIttRU-R+>K1ISvDHLu!rLRk;701q7g5Lm2tpDTY^sgAR{mtGOcf z(1pGM5I|9<+r|i0nf3Drpp-zb0%rO?1bT8F>?=^~rN)7L>s#~tiA&UH#Nraos&(ee z-%tAx%t+_qAlh0n;w?y&gnH4iSgjG;Oy?2Z&$#L?DrTQy0rZ6Aq+RPh)5bHy_hsQV_^gRG zZSZmz*SO+IriO)$ZiM{l%=)D`^0FQkCJK8VY2M@=)HJI?14>Bne_-#&#i+|Ams%HX zU^dcF_H-MAI(YQBqPk%2!W$zyZs}mAe!z?b_&&oRqL!Yr_Jm*(^T!0MfyXUj8i^%3 zxtI#^D+8=dN(u}qR0Um#m9tv1|I}ALEnhy@b0;i2`iak!{{8-VNE|2!VILLdm-Fr5 zld~JW8}!!RX<)104B#1>nVFGY_~L8n5D2cJ`}zBCrz!x@41lLt7#LH40@e>KoJ0xw zjIUZ#0JA*snnqSM+BMq*c~IqfFZXZ*M@d3Q$41L8Fa=c%qbuo#MNHiBzaW~|q5Bgg zMKBKvS9r@#Nrr(=lU?RA5=tV}A2O=pLSWvNSvi=mMGF^c3j2yA0U|i{)zjHKO7uuL zSZHajKoW|~`!g&vUf*jwJ%o)4Aoe3q?u|}#>vbkeO~2Q&kQT)QQI^gSGCPZn)}jA^ z+cJ@ZZvEsWLERnnQV^+>{1fF<=u!RN%it8;36ka;5sT9Y)NAM&7tf;fY6;ERnvP*# zWpaV+Rd20CijNBB+7dcn_@{Zcz6$zG7lxbcVYc8){q>XNc8r-t32e5<2ygdLcEOyh z5a)X;nlJFQ)74Q4Ai2AZiQN|#DRx+6ic{n?vumj|%!oSbd_qKWTt*K=)*gbmr{{Y?{D{P3A!8`1NdOMsGS{SFPjxhK!{?c0;96aXMAvOEr=M5`ija2 zi-wPf2RGi&C6KST5?O8wW)V-b$D{+z1(^Y+9}_SM19`%2JGc@A9+0<##0D^} zU?rt9>9<{`X&D69=(RYBHo>A%YslFoCj*0V;Vm#FzH)yCerJ*_EG&(W=RaAv9hULq zNO@h%K-7nFErV7KnJVm(Y&bGe)aKAXxo<)8s{w1fTmYI!a*j zYJirR{7I*R`@a}$Ru)4lr@gt^*GU|AQ6lEgJK^s+ zj4A9_KLCwE#-M6SY2z_zPwjLk8@tR~zo+<3Pu z2E<_yonlBWV%-bQqxq1u`yKU{bAbo1BL_x^HU^{){;2zIgzc~8=GAS9oM}}#ODUT zjwI&Mf(=hh5t09Dcqa|vc=HU$7M{{C1p~Xa_Az$mtLX9ErZKg=h;caX;IcW{8>Kkg z>(kSgyfc{q2$QADPnR1fs?K$u|^I(} zepI$>1Q}Rq!8j940;sak0}=AqSw1T2^x^_aIwS=Y0`L1vYbw774NzBWfcCg&BtfU?HFNgX_ z>KYn^&nKb2Ec4PKdPS;JlV2qU&gVX*CG?L@QoXYs-)j6ixNRCo*ES|KFjQ&Bk-<=J zZ5}hhvv*D)l!aL>aTXjBg%}g!C<_A-{IE6Qc^tpA?-J~L76Sj$jQiE~hSxIB|N8mm z<)O>pAa_;Aedm*eH4P=fB6?e)MtAMk2JR$>)9Fpo^47}z>r}i0EX!!7(W+pm1Q8LV%aq>bwQkibp;VDU*`0P?b zd#$^cE>r&IFfEiNUY1o3&L-1a^!#0r`1%0GtoBxYr` zgRsA~wY9s8UG};6^nDsj-n1uMi>TCB*Fq$}J&=EU2!#%aU2!t>8QFXO`kv#PSNk59 z;|w5;p9bHQ@=2+M)YBtqY@=plzY4#LFzRwHAVMqgC$`_&{s+i$IF(mndT_4BZBVt+ zJgefoVxb(E_R<2uuM*}@Ql&>*h_zD2LyP|<`QMZ@nR@>huRzn|kvsg~cVH!iT58qs zO6&RGK?R$g^B&BO7@N$mc%IazF@I}a)y^M&*dk*wzd1F=Ds_5un0~e!w9u(p<-P|d zLBH#o+~DgdAk@Imoy;9DaSrf88{C}Z{stgi-RASUAGp*)6~!*Ci{T3b~ov7S5SS|_S;t_@(HBy&!SAU;}(Dd`?&zXN$fM^AVJ?PytGn!D9yxax! zN{#l(&wdB>^pz=*$-Y-st&j45k)^Ke;A}CP?5o~%VFveDt@Tdbu#JR|l^;B)WiP#9 zMofA|89%|6L#{OFn;Y+5i{;R6F_69@qE?NIiUzb@BU%5u0*M=*>)$6hM!|Oemr8{t^QHvj4jK?8)1*-`Ynu=oHvgkMJYak}D{Fiu96kG(`s;Ry7 zt@ppT+SIfiTwVJ=&~~@{pS76#mHO_i$6S}wAP}>u@Z;c zX9yQVL_16-`HU!*xe`3Mg^g@jPdq}yuDyg%u3ibUfKuME#hffV7YN=;#qNx^dmtep zBVRr$HOA?d4c5zWJY5chNjC@&_zOiBQG2Wo|<>+wqr|}jcoWR$z;$moD z5WF9PP^)->nNdqHE5m&5NbJSWQTeCA(l>2i{kc+1FpG_XhN68!dc71ks1J-akjRrl z#r{VBx_`4l6NG$_QZrUD1_-d=r^B^3v%p_c2%J(e4WU^%0P&-!1+$OS;wWWIGKj9& z+A^!#1b%#wv_D&KF5hC#RF6*1Q1&4+L=N=g&8^Rkjg5d1ev1s-QvoI#Fva-p6-3%_ zABQj&Xsa}6ai*rFMf0wPgoJ=mr{Fz^?l?W*$Z-sRU16ZLH~c#6BT~>f?(Ql*|7xz% zeS2O{5K>XkXDEK-!5X(qV#`$}C*RZJQLWkF7QL|vb9w)En*`4J22!PJPQT+I!=;~v za)pI^=$G9$XgGNezb+Ubprv<=wfB3>dM@E(p#8;errL3P9 z78XDr1S&CKs%66gyptt#CeXOw0(~F%x36GZE7&^)rj9#L+Ha$FT(Nn>IslUn4}*Pc zWY*zg_PzONQxuYshP|a4bmKIwKa85l5n7M0neW0Nmm~iaWL{x+A}aOm^)ju{u=L^p z|u3}gKV`%x>AV?$W3G^nC)}VM7RM5?a{_mTnA28=r{D{e8-bb=?0GOZ^&SH>0 zCMG5>vmQP%adljx6W=m#|IR zO?KK65)vOj;`;pWv$BQlwue7N*g%b`Sci^4qv%#mYTu z>!0T?%Bd@&^j#W`Z0lt{t3-rsvwTj8n7+n%mhTqql(o33uRSO1zU5okn^2SxeVd3* zl8gAi0JQ~5`j4Q#{mT6`HuR^w?dTJSPf>oK^u_8b5w~^XARr8t6Syj$9B1CcFs-3?p>z zZgoI-hbVzXp^_0JMu^1PMGguJXZS?bAqzoR2n`KIhnJR?!tsX<8;0CHJw5%AM;;L| z5F6*@Lta0&LA=0GQc_YDELiZ~ zd+&`JHA*B|n>KAigI{{-B{UdO1}YG=ilby?WIXfCGyVJbM}R0aVKjM7Hj2Y9WO`97xZlDd=3%PU8o;`8ar%s*1L9q>Nclhw((2*NAZd|o$ z)zzz45qHp3L>Yu3^hRQ0BJzxV`}W}{Do1U1GmBg0&{(b7W5=khlrCSQ3m0kFu+pAS zIRh;Ibm{4IDA^YEnx%D+?qD+QNhE)Nd(El*V{D0|mrVi&eBtl`eX%n{4@#ECCopt0$YpVeNvuEktS!&*b zz(!$R&5M_bbV;nh3zkeV7?OmEa-8f;ApjPevB>HfszfxiR%NRui z3+^xvM7l$V4t@Xq_b_HS2n+z$lA4+df5iXZy?Z-np3omr<-konE^A$jE zk3IGn{2j|+yIs3>oj!dU{Vz{GM7R^zRuD;$xIh~Skp~t;itqW=4K>1tyk15ezaA12GHB2sw@xJrg{HyAaCKM!tO0y8ZQ3-{gYut${uwLC;6)Z*e|$j!NP=MES!;G1v0S+{N-4hr*yMMgzM zVPBZpl`B_J4~Ta9^y!FAI0Qn;%$YNh#KpzMA)thZha+HRW@g63#6Ytm;vf$IDT2Sh z_~MJm$Vkv9R4lT;PMta(IB)=o;g?^2>D5(IkH!VPX))8b1X{a>diAA@blSB;zaq}o zI!IzVNUqXS`d~UK6}CUWrQ5ZW#*d@$aLUV5BqiwHQ>=gv#Y>FQ;=as`1|hh0`Kr`~<(->cQ9Pt)uJI((RB&7c8;si=tJ*Z3RDjIY0;+p&!% zP7n>F{Yko|v2-Arl9Nm)AUIB(I1#xQJO~L8XbGVZsTC{=rn`Lkau_El z>be6WJzcnPA?!aQBI2i?euC>QTeb|b6_vooK#Ftb%z<&jhPQ6rikz%pzkYBNNDYuE z>=s9fjg5s9)zyqblN5Pk0U5dTttM3^6Nn)gCh($OuScSTtXJdB9RB|W7yzTtlkC^+ R9Jv4h002ovPDHLkV1h>RJR1N2 literal 0 HcmV?d00001 diff --git a/modules/web-console/frontend/public/stylesheets/_bootstrap-custom.scss b/modules/web-console/frontend/public/stylesheets/_bootstrap-custom.scss index 3b528214c9b7c..3dc338f8a0543 100644 --- a/modules/web-console/frontend/public/stylesheets/_bootstrap-custom.scss +++ b/modules/web-console/frontend/public/stylesheets/_bootstrap-custom.scss @@ -58,7 +58,6 @@ @import "../../node_modules/bootstrap-sass/assets/stylesheets/bootstrap/modals"; @import "../../node_modules/bootstrap-sass/assets/stylesheets/bootstrap/tooltip"; @import "../../node_modules/bootstrap-sass/assets/stylesheets/bootstrap/popovers"; -@import "../../node_modules/bootstrap-sass/assets/stylesheets/bootstrap/carousel"; // Utility classes @import "../../node_modules/bootstrap-sass/assets/stylesheets/bootstrap/utilities"; diff --git a/modules/web-console/frontend/views/reset.tpl.pug b/modules/web-console/frontend/views/reset.tpl.pug deleted file mode 100644 index 6bc5b7b0a5b34..0000000000000 --- a/modules/web-console/frontend/views/reset.tpl.pug +++ /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. - -web-console-header - web-console-header-left - ignite-header-title - -.container.body-container - .main-content(ng-controller='resetPassword') - .row - .text-center(ng-if='!token') - p Further instructions for password reset have been sent to your e-mail address. - .text-center(ng-if='error') - p {{::error}} - div(ng-if='token && !error') - form.form-horizontal(name='resetForm' ng-init='reset_info.token = token') - .settings-row - label.col-sm-1 E-mail: - label {{::email}} - .settings-row - label.col-sm-1.required Password: - .col-sm-3 - input#user_password.form-control(ignite-on-enter-focus-move='user_confirm' type='password' ng-model='reset_info.password' placeholder='New password' required) - .settings-row - label.col-sm-1.required Confirm: - .col-sm-3 - input#user_confirm.form-control(type='password' ng-model='reset_info.confirm' ignite-match='reset_info.password' placeholder='Confirm new password' required ignite-on-enter='resetForm.$valid && resetPassword(user_info)') - .settings-row - button.btn.btn-primary(ng-disabled='resetForm.$invalid' ng-click='resetPassword(reset_info)') Reset Password - -web-console-footer diff --git a/modules/web-console/frontend/views/signin.tpl.pug b/modules/web-console/frontend/views/signin.tpl.pug deleted file mode 100644 index 29da8b15b4a6b..0000000000000 --- a/modules/web-console/frontend/views/signin.tpl.pug +++ /dev/null @@ -1,165 +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 lblRequired(txt) - label.col-xs-3.col-md-3.required #{txt} - -web-console-header - web-console-header-left - ignite-header-title - -.container.body-container - .main-content - .row.home - .signin-greedy - .col-xs-12.col-md-6 - form(name='form' novalidate) - .row(style='padding: 0 0 10px 0; margin: 0') - .settings-row(ng-if='action == "signup"') - h3.login-header Sign Up - .settings-row(ng-if='action == "signin"') - h3.login-header Sign In - .settings-row(ng-if='action == "password/forgot"') - h3.login-header Forgot password? - .settings-row - p.col-xs-12.col-md-11(ng-show='action == "password/forgot"') - | That's ok! Simply enter your email below and a reset password link will be sent to you via email. You can then follow that link and select a new password. - .settings-row(ng-show='action == "signin"') - +lblRequired('Email:') - .col-xs-9.col-md-8 - input#signin_email.form-control(ignite-on-enter-focus-move='user_password' type='email' ng-model='ui.email' placeholder='Input email' required) - .settings-row(ng-show='action == "signup"') - +lblRequired('Email:') - .col-xs-9.col-md-8 - input#signup_email.form-control(ignite-on-enter-focus-move='user_password' type='email' ng-model='ui.email' placeholder='Input email' required) - .settings-row(ng-show='action != "password/forgot"') - +lblRequired('Password:') - .col-xs-9.col-md-8 - input#user_password.form-control(ignite-on-enter-focus-move='user_confirm' type='password' ng-model='ui.password' placeholder='Password' ng-required='action != "password/forgot"' ignite-on-enter='action == "signin" && form.$valid && auth(action, ui)') - .settings-row(ng-if='action == "signup"') - +lblRequired('Confirm:') - .col-xs-9.col-md-8 - input#user_confirm.form-control(ignite-on-enter-focus-move='first_name' type='password' ng-model='ui_exclude.confirm' ignite-match='ui.password' placeholder='Confirm password' ng-required='action == "signup"') - .settings-row(ng-show='action == "signup"') - +lblRequired('First Name:') - .col-xs-9.col-md-8 - input#first_name.form-control(ignite-on-enter-focus-move='last_name' type='text' ng-model='ui.firstName' placeholder='Input first name' ng-required='action=="signup"') - .settings-row(ng-show='action == "signup"') - +lblRequired('Last Name:') - .col-xs-9.col-md-8 - input#last_name.form-control(ignite-on-enter-focus-move='company' type='text' ng-model='ui.lastName' placeholder='Input last name' ng-required='action=="signup"') - .settings-row(ng-show='action == "password/forgot"') - +lblRequired('Email:') - .col-xs-9.col-md-8 - input#forgot_email.form-control(ignite-on-enter='form.$valid && forgotPassword(ui)' type='email' ng-model='ui.email' placeholder='Input email' required) - .settings-row(ng-show='action == "signup"') - +lblRequired('Company:') - .col-xs-9.col-md-8 - input#company.form-control(ignite-on-enter-focus-move='country' type='text' ng-model='ui.company' placeholder='Input company name' ng-required='action=="signup"') - .settings-row(ng-show='action == "signup"') - +lblRequired('Country:') - .col-xs-9.col-md-8 - button#country.select-toggle.form-control(ignite-on-enter-focus-move='signup' bs-select bs-options='item.name as item.name for item in countries' type='text' ng-model='ui.country' placeholder='Choose your country' ng-required='action=="signup"') - .settings-row(ignite-terms) - .col-md-offset-3(ng-if='action == "signup" && terms.termsState') - label - input(type='checkbox' ng-model='ui_exclude.agree', ng-required='true') - | I agree to the #[a(ui-sref='{{::terms.termsState}}' target='_blank') terms and conditions] - .col-xs-12.col-md-11 - .login-footer(ng-show='action == "signup"') - a#password-forgot-signup.labelField(ng-click='action = "password/forgot"' ignite-on-click-focus='signin_email') Forgot password? - a.labelLogin(ng-click='action = "signin"' ignite-on-click-focus='signin_email') Sign In - button#signup.btn.btn-primary(ng-click='auth(action, ui)' ng-disabled='form.$invalid') Sign Up - .col-xs-12.col-md-11 - .login-footer(ng-show='action == "password/forgot"') - a.labelField(ng-click='action = "signin"' ignite-on-click-focus='signin_email') Sign In - button#forgot.btn.btn-primary(ng-click='forgotPassword(ui)' ng-disabled='form.$invalid') Send it to me - .col-xs-12.col-md-11 - .login-footer(ng-show='action == "signin"') - a#password-forgot-signin.labelField(ng-click='action = "password/forgot"' ignite-on-click-focus='signin_email') Forgot password? - a.labelLogin(ng-click='action = "signup"' ignite-on-click-focus='first_name') Sign Up - button#login.btn.btn-primary(ng-click='auth(action, ui)' ng-disabled='form.$invalid') Sign In - - .col-xs-12.col-md-11.home-panel - ignite-features - .col-xs-12.col-md-6 - #carousel.carousel.slide - //- Indicators - ol.carousel-indicators - li.active(data-target='#carousel', data-slide-to='0') - li(data-target='#carousel', data-slide-to='1') - li(data-target='#carousel', data-slide-to='2') - li(data-target='#carousel', data-slide-to='3') - li(data-target='#carousel', data-slide-to='4') - li(data-target='#carousel', data-slide-to='5') - li(data-target='#carousel', data-slide-to='6') - li(data-target='#carousel', data-slide-to='7') - //- Wrapper for slides - .carousel-inner(role='listbox') - .item.active - img(src='/images/cluster.png', alt='Clusters screen') - .carousel-caption - h3 Clusters screen - p Configure clusters, link clusters to caches - .item - img(src='/images/cache.png', alt='Caches screen') - .carousel-caption - h3 Caches screen - p Configure caches, link domain models to caches, link caches to clusters - .item - img(src='/images/domains.png', alt='Domain model screen') - .carousel-caption - h3 Domain model screen - p Manually enter domain model or import from database - .item - img(src='/images/summary.png', alt='Summary screen') - .carousel-caption - h3 Summary screen - p Preview XML config, JAVA code,Docker file and download project - .item - img(src='/images/query-table.png', alt='Query') - .carousel-caption - h3 Query - p Explain SQL, execute, scan queries - .item - img(src='/images/query-metadata.png', alt='Cache metadata') - .carousel-caption - h3 Cache metadata - p View cache type metadata - .item - img(src='/images/query-chart.png', alt='Query chart') - .carousel-caption - h3 Query chart - p View data in tabular form and as charts - .item - img(src='/images/multicluster.png', alt='Multicluster support') - .carousel-caption - h3 Multicluster support - p Execute queries on different clusters - //- Controls - a.left.carousel-control(href='#carousel', ng-click='$event.preventDefault()', role='button', data-slide='prev') - span.fa.fa-chevron-left(aria-hidden='true') - span.sr-only Previous - a.right.carousel-control(href='#carousel', ng-click='$event.preventDefault()', role='button', data-slide='next') - span.fa.fa-chevron-right(aria-hidden='true') - span.sr-only Next - -web-console-footer - -script. - $('.carousel').carousel() - - From 679aeca7a3ff60a9dd478966d3949107d302d5db Mon Sep 17 00:00:00 2001 From: Andrey Novikov Date: Mon, 19 Feb 2018 14:56:07 +0700 Subject: [PATCH 046/314] IGNITE-7650 Fixed headers. (cherry picked from commit 67922b3) --- .../page-password-changed/controller.js | 20 +++++++++++++------ .../components/page-password-changed/index.js | 20 +++++++++++++------ .../page-password-changed/style.scss | 20 +++++++++++++------ .../page-password-changed/template.pug | 20 +++++++++++++------ 4 files changed, 56 insertions(+), 24 deletions(-) diff --git a/modules/web-console/frontend/app/components/page-password-changed/controller.js b/modules/web-console/frontend/app/components/page-password-changed/controller.js index 5b8301d20771d..e3b0a035b5e14 100644 --- a/modules/web-console/frontend/app/components/page-password-changed/controller.js +++ b/modules/web-console/frontend/app/components/page-password-changed/controller.js @@ -1,10 +1,18 @@ /* - * Copyright (C) GridGain Systems. All Rights Reserved. - * _________ _____ __________________ _____ - * __ ____/___________(_)______ /__ ____/______ ____(_)_______ - * _ / __ __ ___/__ / _ __ / _ / __ _ __ `/__ / __ __ \ - * / /_/ / _ / _ / / /_/ / / /_/ / / /_/ / _ / _ / / / - * \____/ /_/ /_/ \_,__/ \____/ \__,_/ /_/ /_/ /_/ + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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 { diff --git a/modules/web-console/frontend/app/components/page-password-changed/index.js b/modules/web-console/frontend/app/components/page-password-changed/index.js index 4d624a73e8c36..7c244beff0622 100644 --- a/modules/web-console/frontend/app/components/page-password-changed/index.js +++ b/modules/web-console/frontend/app/components/page-password-changed/index.js @@ -1,10 +1,18 @@ /* - * Copyright (C) GridGain Systems. All Rights Reserved. - * _________ _____ __________________ _____ - * __ ____/___________(_)______ /__ ____/______ ____(_)_______ - * _ / __ __ ___/__ / _ __ / _ / __ _ __ `/__ / __ __ \ - * / /_/ / _ / _ / / /_/ / / /_/ / / /_/ / _ / _ / / / - * \____/ /_/ /_/ \_,__/ \____/ \__,_/ /_/ /_/ /_/ + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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'; diff --git a/modules/web-console/frontend/app/components/page-password-changed/style.scss b/modules/web-console/frontend/app/components/page-password-changed/style.scss index 944fc47534340..c4f3d83c9fb50 100644 --- a/modules/web-console/frontend/app/components/page-password-changed/style.scss +++ b/modules/web-console/frontend/app/components/page-password-changed/style.scss @@ -1,10 +1,18 @@ /* - * Copyright (C) GridGain Systems. All Rights Reserved. - * _________ _____ __________________ _____ - * __ ____/___________(_)______ /__ ____/______ ____(_)_______ - * _ / __ __ ___/__ / _ __ / _ / __ _ __ `/__ / __ __ \ - * / /_/ / _ / _ / / /_/ / / /_/ / / /_/ / _ / _ / / / - * \____/ /_/ /_/ \_,__/ \____/ \__,_/ /_/ /_/ /_/ + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ page-password-changed { diff --git a/modules/web-console/frontend/app/components/page-password-changed/template.pug b/modules/web-console/frontend/app/components/page-password-changed/template.pug index 33f20d4f41bbe..7d2d986c812b2 100644 --- a/modules/web-console/frontend/app/components/page-password-changed/template.pug +++ b/modules/web-console/frontend/app/components/page-password-changed/template.pug @@ -1,10 +1,18 @@ //- - Copyright (C) GridGain Systems. All Rights Reserved. - _________ _____ __________________ _____ - __ ____/___________(_)______ /__ ____/______ ____(_)_______ - _ / __ __ ___/__ / _ __ / _ / __ _ __ `/__ / __ __ \ - / /_/ / _ / _ / / /_/ / / /_/ / / /_/ / _ / _ / / / - \____/ /_/ /_/ \_,__/ \____/ \__,_/ /_/ /_/ /_/ + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. .container.body-container h2 Ready! From 5d5a6a05ec49f895e8a5edd505e496dcfe58a208 Mon Sep 17 00:00:00 2001 From: Vasiliy Sisko Date: Wed, 21 Feb 2018 11:21:02 +0700 Subject: [PATCH 047/314] IGNITE-6094 Web Agent: Enabled persistent in demo mode. (cherry picked from commit 3c35900) --- .../ignite/console/demo/AgentClusterDemo.java | 87 +++++++++++++++---- 1 file changed, 71 insertions(+), 16 deletions(-) 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 6bfc5bd16a257..1fd286c3a5247 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 @@ -17,6 +17,8 @@ package org.apache.ignite.console.demo; +import java.io.File; +import java.nio.file.Paths; import java.util.Collection; import java.util.Collections; import java.util.concurrent.CountDownLatch; @@ -24,9 +26,16 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteServices; import org.apache.ignite.Ignition; +import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.DataRegionConfiguration; import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; @@ -37,15 +46,14 @@ import org.apache.ignite.console.demo.service.DemoServiceKeyAffinity; import org.apache.ignite.console.demo.service.DemoServiceMultipleInstances; import org.apache.ignite.console.demo.service.DemoServiceNodeSingleton; -import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.cache.persistence.filename.PdsConsistentIdProcessor; import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.logger.slf4j.Slf4jLogger; 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.eventstorage.memory.MemoryEventStorageSpi; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import static org.apache.ignite.IgniteSystemProperties.IGNITE_ATOMIC_CACHE_DELETE_HISTORY_SIZE; import static org.apache.ignite.IgniteSystemProperties.IGNITE_JETTY_PORT; @@ -54,11 +62,12 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_QUIET; import static org.apache.ignite.IgniteSystemProperties.IGNITE_UPDATE_NOTIFIER; import static org.apache.ignite.configuration.DataStorageConfiguration.DFLT_DATA_REGION_INITIAL_SIZE; +import static org.apache.ignite.configuration.WALMode.LOG_ONLY; import static org.apache.ignite.console.demo.AgentDemoUtils.newScheduledThreadPool; import static org.apache.ignite.events.EventType.EVTS_DISCOVERY; -import static org.apache.ignite.internal.visor.util.VisorTaskUtils.VISOR_TASK_EVTS; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_REST_JETTY_ADDRS; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_REST_JETTY_PORT; +import static org.apache.ignite.internal.visor.util.VisorTaskUtils.VISOR_TASK_EVTS; /** * Demo for cluster features like SQL and Monitoring. @@ -73,27 +82,46 @@ public class AgentClusterDemo { private static final AtomicBoolean initGuard = new AtomicBoolean(); /** */ - private static CountDownLatch initLatch = new CountDownLatch(1); + private static final String SRV_NODE_NAME = "demo-server-"; /** */ - private static volatile String demoUrl; + private static final String CLN_NODE_NAME = "demo-client-"; /** */ private static final int NODE_CNT = 3; + /** */ + private static final int WAL_SEGMENTS = 5; + + /** WAL file segment size, 16MBytes. */ + private static final int WAL_SEGMENT_SZ = 16 * 1024 * 1024; + + /** */ + private static CountDownLatch initLatch = new CountDownLatch(1); + + /** */ + private static volatile String demoUrl; + /** * Configure node. + * * @param basePort Base port. * @param gridIdx Ignite instance name index. * @param client If {@code true} then start client node. * @return IgniteConfiguration */ - private static IgniteConfiguration igniteConfiguration(int basePort, int gridIdx, boolean client) { + private static IgniteConfiguration igniteConfiguration(int basePort, int gridIdx, boolean client) + throws IgniteCheckedException { IgniteConfiguration cfg = new IgniteConfiguration(); - cfg.setIgniteInstanceName((client ? "demo-client-" : "demo-server-" ) + gridIdx); + cfg.setIgniteInstanceName((client ? CLN_NODE_NAME : SRV_NODE_NAME) + gridIdx); cfg.setLocalHost("127.0.0.1"); cfg.setEventStorageSpi(new MemoryEventStorageSpi()); + cfg.setConsistentId(cfg.getIgniteInstanceName()); + + File workDir = new File(U.workDirectory(null, null), "demo-work"); + + cfg.setWorkDirectory(workDir.getAbsolutePath()); int[] evts = new int[EVTS_DISCOVERY.length + VISOR_TASK_EVTS.length]; @@ -137,11 +165,18 @@ private static IgniteConfiguration igniteConfiguration(int basePort, int gridIdx dataRegCfg.setName("demo"); dataRegCfg.setMetricsEnabled(true); dataRegCfg.setMaxSize(DFLT_DATA_REGION_INITIAL_SIZE); + dataRegCfg.setPersistenceEnabled(true); DataStorageConfiguration dataStorageCfg = new DataStorageConfiguration(); + dataStorageCfg.setMetricsEnabled(true); + dataStorageCfg.setStoragePath(PdsConsistentIdProcessor.DB_DEFAULT_FOLDER); dataStorageCfg.setDefaultDataRegionConfiguration(dataRegCfg); dataStorageCfg.setSystemRegionMaxSize(DFLT_DATA_REGION_INITIAL_SIZE); + dataStorageCfg.setWalMode(LOG_ONLY); + dataStorageCfg.setWalSegments(WAL_SEGMENTS); + dataStorageCfg.setWalSegmentSize(WAL_SEGMENT_SZ); + cfg.setDataStorageConfiguration(dataStorageCfg); if (client) @@ -197,23 +232,37 @@ public static CountDownLatch tryStart() { int idx = cnt.incrementAndGet(); int port = basePort.get(); - IgniteEx ignite = null; + boolean first = idx == 0; try { - ignite = (IgniteEx)Ignition.start(igniteConfiguration(port, idx, false)); + IgniteConfiguration cfg = igniteConfiguration(port, idx, false); - if (idx == 0) { - Collection jettyAddrs = ignite.localNode().attribute(ATTR_REST_JETTY_ADDRS); + if (first) { + U.delete(Paths.get(cfg.getWorkDirectory())); + + U.resolveWorkDirectory( + cfg.getWorkDirectory(), + cfg.getDataStorageConfiguration().getStoragePath(), + true + ); + } + + Ignite ignite = Ignition.start(cfg); + + if (first) { + ClusterNode node = ignite.cluster().localNode(); + + Collection jettyAddrs = node.attribute(ATTR_REST_JETTY_ADDRS); if (jettyAddrs == null) { - ignite.cluster().stopNodes(); + Ignition.stopAll(true); throw new IgniteException("DEMO: Failed to start Jetty REST server on embedded node"); } String jettyHost = jettyAddrs.iterator().next(); - Integer jettyPort = ignite.localNode().attribute(ATTR_REST_JETTY_PORT); + Integer jettyPort = node.attribute(ATTR_REST_JETTY_PORT); if (F.isEmpty(jettyHost) || jettyPort == null) throw new IgniteException("DEMO: Failed to start Jetty REST handler on embedded node"); @@ -226,7 +275,7 @@ public static CountDownLatch tryStart() { } } catch (Throwable e) { - if (idx == 0) { + if (first) { basePort.getAndAdd(50); log.warn("DEMO: Failed to start embedded node.", e); @@ -236,7 +285,13 @@ public static CountDownLatch tryStart() { } finally { if (idx == NODE_CNT) { - deployServices(ignite.services(ignite.cluster().forServers())); + Ignite ignite = Ignition.ignite(SRV_NODE_NAME + 0); + + if (ignite != null) { + ignite.cluster().active(true); + + deployServices(ignite.services(ignite.cluster().forServers())); + } log.info("DEMO: All embedded nodes for demo successfully started"); From e35d8cfb06f52765959fc2e1883bf70fe0259f45 Mon Sep 17 00:00:00 2001 From: Alexander Kalinin Date: Wed, 21 Feb 2018 14:03:20 +0700 Subject: [PATCH 048/314] IGNITE-7320 Web Console - Fixed table headers for Safari. (cherry picked from commit 04a025b) --- .../web-console/frontend/app/primitives/ui-grid/index.scss | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/modules/web-console/frontend/app/primitives/ui-grid/index.scss b/modules/web-console/frontend/app/primitives/ui-grid/index.scss index 331b075be8851..54f444f5631d8 100644 --- a/modules/web-console/frontend/app/primitives/ui-grid/index.scss +++ b/modules/web-console/frontend/app/primitives/ui-grid/index.scss @@ -214,7 +214,7 @@ position: absolute; top: 0; - right: 9px; + right: 15px; z-index: 1000; width: 5px; @@ -507,6 +507,10 @@ .ui-grid-selection-row-header-buttons::before { opacity: 1; } + + .ui-grid-clearfix:before, .ui-grid-clearfix:after { + display: flex; + } } .ui-grid--ignite.ui-grid-disabled-group-selection { From 20cb1439f48b9a3c985f65784af36ef2c6f45e4f Mon Sep 17 00:00:00 2001 From: Andrey Novikov Date: Thu, 22 Feb 2018 09:54:05 +0700 Subject: [PATCH 049/314] IGNITE-7650 Fixed counties codes. (cherry picked from commit fc40261) --- .../web-console/frontend/app/services/Countries.service.js | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/modules/web-console/frontend/app/services/Countries.service.js b/modules/web-console/frontend/app/services/Countries.service.js index 5ad3e6a63f96c..575951ec136a8 100644 --- a/modules/web-console/frontend/app/services/Countries.service.js +++ b/modules/web-console/frontend/app/services/Countries.service.js @@ -18,8 +18,8 @@ import COUNTRIES from 'app/data/countries.json'; export default ['IgniteCountries', function() { - const indexByName = _.keyBy(COUNTRIES, 'name'); - const UNDEFINED_COUNTRY = {name: '', code: ''}; + const indexByName = _.keyBy(COUNTRIES, 'label'); + const UNDEFINED_COUNTRY = {label: '', value: '', code: ''}; const getByName = (name) => (indexByName[name] || UNDEFINED_COUNTRY); const getAll = () => (COUNTRIES); From 50d1389cd60e148005701e3450c8ae9998fd5fc9 Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Thu, 22 Feb 2018 10:28:32 +0700 Subject: [PATCH 050/314] IGNITE-6287 Web Console: Fixed regression for EXPLAIN. (cherry picked from commit 2bee8f8) --- .../frontend/app/components/page-queries/controller.js | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/modules/web-console/frontend/app/components/page-queries/controller.js b/modules/web-console/frontend/app/components/page-queries/controller.js index a871a0d414660..7d03a36a38dae 100644 --- a/modules/web-console/frontend/app/components/page-queries/controller.js +++ b/modules/web-console/frontend/app/components/page-queries/controller.js @@ -1462,6 +1462,10 @@ export default class { return false; }; + $scope.cacheNameForSql = (paragraph) => { + return $scope.ddlAvailable(paragraph) && !paragraph.useAsDefaultSchema ? null : paragraph.cacheName; + }; + $scope.execute = (paragraph, local = false) => { const nonCollocatedJoins = !!paragraph.nonCollocatedJoins; const enforceJoinOrder = !!paragraph.enforceJoinOrder; @@ -1481,7 +1485,7 @@ export default class { .then(() => { const args = paragraph.queryArgs = { type: 'QUERY', - cacheName: ($scope.ddlAvailable(paragraph) && !paragraph.useAsDefaultSchema) ? null : paragraph.cacheName, + cacheName: $scope.cacheNameForSql(paragraph), query: paragraph.query, pageSize: paragraph.pageSize, maxPages: paragraph.maxPages, @@ -1541,7 +1545,7 @@ export default class { .then((nid) => { const args = paragraph.queryArgs = { type: 'EXPLAIN', - cacheName: paragraph.cacheName, + cacheName: $scope.cacheNameForSql(paragraph), query: 'EXPLAIN ' + paragraph.query, pageSize: paragraph.pageSize }; From 330e502d5effea485254151ebf9b666e67d68475 Mon Sep 17 00:00:00 2001 From: Alexander Kalinin Date: Mon, 26 Feb 2018 15:13:52 +0700 Subject: [PATCH 051/314] IGNITE-7729. Web Console. Added roles support in E2E tests. (cherry picked from commit a83b264) --- modules/web-console/e2e/testcafe/envtools.js | 25 ++++--- .../e2e/testcafe/fixtures/admin-panel.js | 32 +++++---- .../web-console/e2e/testcafe/fixtures/auth.js | 66 +++++++++++-------- .../e2e/testcafe/fixtures/menu-smoke.js | 21 +++--- .../fixtures/user-profile/credentials.js | 27 ++++---- .../testcafe/fixtures/user-profile/profile.js | 28 ++++---- .../e2e/testcafe/page-models/PageSignIn.js | 37 +++++++++++ modules/web-console/e2e/testcafe/roles.js | 47 ++++--------- modules/web-console/e2e/testcafe/testcafe.js | 8 +-- 9 files changed, 163 insertions(+), 128 deletions(-) create mode 100644 modules/web-console/e2e/testcafe/page-models/PageSignIn.js diff --git a/modules/web-console/e2e/testcafe/envtools.js b/modules/web-console/e2e/testcafe/envtools.js index c251b5f094590..06a3803e7c807 100644 --- a/modules/web-console/e2e/testcafe/envtools.js +++ b/modules/web-console/e2e/testcafe/envtools.js @@ -35,7 +35,7 @@ const insertTestUser = ({userId = '000000000000000000000001', token = 'ppw4tPI3J throw err; } - // add user + // Add test user. const user = { _id: objectid(userId), salt: 'ca8b49c2eacd498a0973de30c0873c166ed99fa0605981726aedcc85bee17832', @@ -50,12 +50,12 @@ const insertTestUser = ({userId = '000000000000000000000001', token = 'ppw4tPI3J admin: true, token, attempts: 0, - lastLogin: '2016-06-28T10:41:07.463Z', + lastLogin: '2018-01-28T10:41:07.463Z', resetPasswordToken: '892rnLbEnVp1FP75Jgpi' }; db.collection('accounts').insert(user); - // add spaces + // Add test spaces. const spaces = [ { @@ -80,7 +80,7 @@ const insertTestUser = ({userId = '000000000000000000000001', token = 'ppw4tPI3J }); }; -const removeData = () => { +const dropTestDB = () => { return new Promise((resolve, reject) => { MongoClient.connect(mongoUrl, async(err, db) => { if (err) @@ -149,9 +149,9 @@ const startEnv = () => { const command = `${process.platform === 'win32' ? 'npm.cmd' : 'npm'} start`; let port = 9001; - if (process.env.APP_URL) { - port = parseInt(url.parse(process.env.APP_URL).port) || 80; - } + + if (process.env.APP_URL) + port = parseInt(url.parse(process.env.APP_URL).port, 10) || 80; const backendInstanceLaunch = exec(command, 'Start listening', '../../backend', {server_port: 3001, mongodb_url: mongoUrl}); const frontendInstanceLaunch = exec(command, 'Compiled successfully', '../../frontend', {BACKEND_PORT: 3001, PORT: port}); @@ -172,16 +172,21 @@ if (start) { startEnv(); process.on('SIGINT', async() => { - await removeData(); + await dropTestDB(); process.exit(0); }); } if (stop) { - removeData(); + dropTestDB(); console.log('Cleaning done...'); } -module.exports = { startEnv, removeData, insertTestUser }; + +const resolveUrl = (targetUrl) => { + return url.resolve(process.env.APP_URL || 'http://localhost:9001', targetUrl); +}; + +module.exports = { startEnv, insertTestUser, dropTestDB, resolveUrl }; diff --git a/modules/web-console/e2e/testcafe/fixtures/admin-panel.js b/modules/web-console/e2e/testcafe/fixtures/admin-panel.js index 39c596e2284bb..9baf055489eff 100644 --- a/modules/web-console/e2e/testcafe/fixtures/admin-panel.js +++ b/modules/web-console/e2e/testcafe/fixtures/admin-panel.js @@ -15,22 +15,24 @@ * limitations under the License. */ -const { Selector } = require('testcafe'); -const { removeData, insertTestUser } = require('../envtools'); -const { signIn } = require('../roles'); +import { Selector } from 'testcafe'; +import { AngularJSSelector } from 'testcafe-angular-selectors'; +import { dropTestDB, insertTestUser, resolveUrl } from '../envtools'; +import { createRegularUser } from '../roles'; + +const regularUser = createRegularUser(); fixture('Checking admin panel') - .page `${process.env.APP_URL || 'http://localhost:9001/'}settings/admin` - .beforeEach(async(t) => { - await t.setNativeDialogHandler(() => true); - await removeData(); + .before(async() => { + await dropTestDB(); await insertTestUser(); - await signIn(t); - - await t.navigateTo(`${process.env.APP_URL || 'http://localhost:9001/'}settings/admin`); + }) + .beforeEach(async(t) => { + await t.useRole(regularUser); + await t.navigateTo(resolveUrl('/settings/admin')); }) .after(async() => { - await removeData(); + await dropTestDB(); }); test('Testing setting notifications', async(t) => { @@ -41,20 +43,22 @@ test('Testing setting notifications', async(t) => { .ok() .click('.ace_content') .pressKey('t e s t space m e s s a g e') + .click(AngularJSSelector.byModel('$ctrl.isShown')) .click('#btn-submit'); await t - .expect(Selector('div').withText('test message').exists) - .ok(); + .expect(Selector('.wch-notification').innerText) + .eql('test message'); await t.click(Selector('button').withAttribute('ng-click', 'ctrl.changeUserNotifications()')); await t .click('.ace_content') .pressKey('ctrl+a delete') + .click(AngularJSSelector.byModel('$ctrl.isShown')) .click('#btn-submit'); await t - .expect(Selector('div').withText('test message').exists) + .expect(Selector('.wch-notification', { visibilityCheck: false } ).visible) .notOk(); }); diff --git a/modules/web-console/e2e/testcafe/fixtures/auth.js b/modules/web-console/e2e/testcafe/fixtures/auth.js index c84556063dc65..9267f5e7efff1 100644 --- a/modules/web-console/e2e/testcafe/fixtures/auth.js +++ b/modules/web-console/e2e/testcafe/fixtures/auth.js @@ -15,30 +15,29 @@ * limitations under the License. */ -const { Selector, Role } = require('testcafe'); -const { signUp } = require('../roles'); -const { AngularJSSelector } = require('testcafe-angular-selectors'); -const { removeData, insertTestUser } = require('../envtools'); +import { Selector } from 'testcafe'; +import { AngularJSSelector } from 'testcafe-angular-selectors'; +import { dropTestDB, insertTestUser, resolveUrl } from '../envtools'; +import { createRegularUser } from '../roles'; fixture('Checking Ignite auth screen') - .page `${process.env.APP_URL || 'http://localhost:9001/'}signin` + .page(resolveUrl('/signin')) .beforeEach(async(t) => { - await removeData(); - + await dropTestDB(); await t.setNativeDialogHandler(() => true); - await t.useRole(Role.anonymous()); }) .after(async() => { - await removeData(); + await dropTestDB(); }); test('Testing Ignite signup validation and signup success', async(t) => { async function checkBtnDisabled() { - const btnDisabled = await t.expect(Selector('#signup_submit').getAttribute('disabled')).ok(); + // Timeout is used to force Selector fetching without long waiting for default timeout (3000 ms) + const btnDisabled = await t.expect(Selector('#signup_submit', {timeout: 1000}).getAttribute('disabled')).ok(); const btnNotWorks = await t .click('#signup_submit') - .expect(Selector('title').innerText).eql('Apache Ignite - Management Tool and Configuration Wizard – Apache Ignite Web Console'); + .expect(Selector('title', {timeout: 1000}).innerText).eql('Apache Ignite - Management Tool and Configuration Wizard – Apache Ignite Web Console'); return btnDisabled && btnNotWorks; } @@ -82,12 +81,26 @@ test('Testing Ignite signup validation and signup success', async(t) => { }); -test('Testing Ignite validation and successful sign in of existing user', async(t) => { +test('Testing logout', async(t) => { + await insertTestUser(); + + const user = createRegularUser(); + + await t.useRole(user); + + await t.click(Selector('div').withAttribute('bs-dropdown', 'userbar.items')); + await t + .click(Selector('a').withAttribute('ui-sref', 'logout')) + .expect(Selector('title').innerText).eql('Apache Ignite - Management Tool and Configuration Wizard – Apache Ignite Web Console'); +}); + +test('Testing Ignite validation for sign in of existing user', async(t) => { async function checkSignInBtnDisabled() { - const btnDisabled = await t.expect(await Selector('#signin_submit').getAttribute('disabled')).ok(); + // Timeout is used to force Selector fetching without long waiting for default timeout (3000 ms) + const btnDisabled = await t.expect(Selector('#signin_submit', {timeout: 1000}).getAttribute('disabled')).ok(); const btnNotWorks = await t .click('#signin_submit') - .expect(Selector('title').innerText).eql('Apache Ignite - Management Tool and Configuration Wizard – Apache Ignite Web Console'); + .expect(Selector('title', {timeout: 1000}).innerText).eql('Apache Ignite - Management Tool and Configuration Wizard – Apache Ignite Web Console'); return btnDisabled && btnNotWorks; } @@ -96,7 +109,7 @@ test('Testing Ignite validation and successful sign in of existing user', async( // Checking signin validation. await t - .typeText(AngularJSSelector.byModel('ui.email'), 'test@test.com'); + .typeText(AngularJSSelector.byModel('ui.email'), 'a@a'); await checkSignInBtnDisabled(); await t @@ -109,22 +122,24 @@ test('Testing Ignite validation and successful sign in of existing user', async( .pressKey('ctrl+a delete') .typeText(AngularJSSelector.byModel('ui.email'), 'testtest.com'); await checkSignInBtnDisabled(); +}); + - // Checking regular sigin in +test('Testing successfull signin', async(t) => { + await insertTestUser(); + + // Checking regular signing in await t .click(AngularJSSelector.byModel('ui.email')) .pressKey('ctrl+a delete') .typeText(AngularJSSelector.byModel('ui.email'), 'a@a') - .click(AngularJSSelector.byModel('ui.password')) - .pressKey('ctrl+a delete') .typeText(AngularJSSelector.byModel('ui.password'), 'a') .click('#signin_submit') .expect(Selector('title').innerText).eql('Basic Configuration – Apache Ignite Web Console'); - }); test('Forbid Ignite signing up of already existing user', async(t) => { - await insertTestUser(); + insertTestUser(); await t .typeText(AngularJSSelector.byModel('ui_signup.email'), 'a@a') @@ -142,6 +157,8 @@ test('Forbid Ignite signing up of already existing user', async(t) => { }); test('Test Ignite password reset', async(t) => { + await insertTestUser(); + await t.click(Selector('#forgot_show')); // Testing incorrect email. @@ -167,12 +184,3 @@ test('Test Ignite password reset', async(t) => { await t.expect(Selector('#popover-validation-message').withText('Account with that email address does not exists!').exists).notOk(); }); - -test('Testing Ignite loguout', async(t) => { - await signUp(t); - - await t.click(Selector('div').withAttribute('bs-dropdown', 'userbar.items')); - await t - .click(Selector('a').withAttribute('ui-sref', 'logout')) - .expect(Selector('title').innerText).eql('Apache Ignite - Management Tool and Configuration Wizard – Apache Ignite Web Console'); -}); diff --git a/modules/web-console/e2e/testcafe/fixtures/menu-smoke.js b/modules/web-console/e2e/testcafe/fixtures/menu-smoke.js index 9ae79b4ee30a9..97ef661fcfe5b 100644 --- a/modules/web-console/e2e/testcafe/fixtures/menu-smoke.js +++ b/modules/web-console/e2e/testcafe/fixtures/menu-smoke.js @@ -15,23 +15,26 @@ * limitations under the License. */ -const { Selector } = require('testcafe'); -const { removeData } = require('../envtools'); -const { signUp } = require('../roles'); +import { Selector } from 'testcafe'; +import { dropTestDB, insertTestUser, resolveUrl } from '../envtools'; +import { createRegularUser } from '../roles'; + +const regularUser = createRegularUser(); fixture('Checking Ingite main menu') - .page `${process.env.APP_URL || 'http://localhost:9001/'}` + .before(async() => { + await dropTestDB(); + await insertTestUser(); + }) .beforeEach(async(t) => { - await t.setNativeDialogHandler(() => true); - await removeData(); - await signUp(t); + await t.useRole(regularUser); + await t.navigateTo(resolveUrl('/')); }) .after(async() => { - await removeData(); + await dropTestDB(); }); test('Ingite main menu smoke test', async(t) => { - await t .click(Selector('a').withAttribute('ui-sref', 'base.configuration.tabs')) .expect(Selector('title').innerText) diff --git a/modules/web-console/e2e/testcafe/fixtures/user-profile/credentials.js b/modules/web-console/e2e/testcafe/fixtures/user-profile/credentials.js index 491a10026d349..33d22326328ed 100644 --- a/modules/web-console/e2e/testcafe/fixtures/user-profile/credentials.js +++ b/modules/web-console/e2e/testcafe/fixtures/user-profile/credentials.js @@ -15,22 +15,23 @@ * limitations under the License. */ -const { Selector } = require('testcafe'); -const { removeData, insertTestUser } = require('../../envtools'); -const { signIn } = require('../../roles'); +import { Selector } from 'testcafe'; +import { dropTestDB, insertTestUser, resolveUrl } from '../../envtools'; +import { createRegularUser } from '../../roles'; + +const regularUser = createRegularUser(); fixture('Checking user credentials change') - .page `${process.env.APP_URL || 'http://localhost:9001/'}settings/profile` - .beforeEach(async(t) => { - await t.setNativeDialogHandler(() => true); - await removeData(); + .before(async() => { + await dropTestDB(); await insertTestUser(); - await signIn(t); - - await t.navigateTo(`${process.env.APP_URL || 'http://localhost:9001/'}settings/profile`); + }) + .beforeEach(async(t) => { + await t.useRole(regularUser); + await t.navigateTo(resolveUrl('/settings/profile')); }) .after(async() => { - await removeData(); + await dropTestDB(); }); test('Testing secure token change', async(t) => { @@ -42,7 +43,7 @@ test('Testing secure token change', async(t) => { .click(Selector('i').withAttribute('ng-click', '$ctrl.generateToken()')) .expect(Selector('p').withText('Are you sure you want to change security token?').exists) .ok() - .click('#confirm-btn-ok', {timeout: 5000}); + .click('#confirm-btn-ok'); await t .expect(await Selector('#current-security-token').innerText) @@ -60,4 +61,4 @@ test('Testing password change', async(t) => { await t .expect(Selector('span').withText('Profile saved.').exists) .ok(); -}); \ No newline at end of file +}); diff --git a/modules/web-console/e2e/testcafe/fixtures/user-profile/profile.js b/modules/web-console/e2e/testcafe/fixtures/user-profile/profile.js index 738d596bf88f5..398b0ada2d240 100644 --- a/modules/web-console/e2e/testcafe/fixtures/user-profile/profile.js +++ b/modules/web-console/e2e/testcafe/fixtures/user-profile/profile.js @@ -15,26 +15,26 @@ * limitations under the License. */ -const { Selector } = require('testcafe'); -const { removeData, insertTestUser } = require('../../envtools'); -const { signIn } = require('../../roles'); +import { Selector } from 'testcafe'; +import { dropTestDB, insertTestUser, resolveUrl } from '../../envtools'; +import { createRegularUser } from '../../roles'; + +const regularUser = createRegularUser(); fixture('Checking user profile') - .page `${process.env.APP_URL || 'http://localhost:9001/'}settings/profile` - .beforeEach(async(t) => { - await t.setNativeDialogHandler(() => true); - await removeData(); + .before(async() => { + await dropTestDB(); await insertTestUser(); - await signIn(t); - - await t.navigateTo(`${process.env.APP_URL || 'http://localhost:9001/'}settings/profile`); + }) + .beforeEach(async(t) => { + await t.useRole(regularUser); + await t.navigateTo(resolveUrl('/settings/profile')); }) .after(async() => { - await removeData(); + await dropTestDB(); }); test('Testing user data change', async(t) => { - const newUserData = { firstName: { selector: '#firstNameInput', @@ -70,7 +70,7 @@ test('Testing user data change', async(t) => { .click(Selector('span').withText(newUserData.country.value)) .click(Selector('button').withText('Save Changes')); - await t.navigateTo(`${process.env.APP_URL || 'http://localhost:9001/'}settings/profile`); + await t.navigateTo(resolveUrl('/settings/profile')); ['firstName', 'lastName', 'email', 'company'].forEach(async(item) => { await t @@ -81,4 +81,4 @@ test('Testing user data change', async(t) => { await t .expect(Selector(newUserData.country.selector).innerText) .eql(newUserData.country.value); -}); \ No newline at end of file +}); diff --git a/modules/web-console/e2e/testcafe/page-models/PageSignIn.js b/modules/web-console/e2e/testcafe/page-models/PageSignIn.js new file mode 100644 index 0000000000000..1939d90b0d1a6 --- /dev/null +++ b/modules/web-console/e2e/testcafe/page-models/PageSignIn.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. + */ + +import {Selector, t} from 'testcafe'; +import { resolveUrl } from '../envtools'; +import {AngularJSSelector} from 'testcafe-angular-selectors'; + +export class PageSignIn { + async open() { + await t.navigateTo(resolveUrl('/signin')); + + this.inputLoginEmail = AngularJSSelector.byModel('ui.email'); + this.inputLoginPassword = AngularJSSelector.byModel('ui.password'); + this.signinButton = Selector('#signin_submit'); + } + + async login(email, password) { + return await t + .typeText(this.inputLoginEmail, email) + .typeText(this.inputLoginPassword, password) + .click(this.signinButton); + } +} diff --git a/modules/web-console/e2e/testcafe/roles.js b/modules/web-console/e2e/testcafe/roles.js index 568a520fec0e1..99a4d313dd6fc 100644 --- a/modules/web-console/e2e/testcafe/roles.js +++ b/modules/web-console/e2e/testcafe/roles.js @@ -15,42 +15,19 @@ * limitations under the License. */ -const path = require('path'); -const { Selector } = require('testcafe'); -const { AngularJSSelector } = require('testcafe-angular-selectors'); +const { Role, t } = require('testcafe'); +import { resolveUrl } from './envtools'; +const { PageSignIn } = require('./page-models/PageSignIn'); -const igniteSignUp = async(t) => { - await t.navigateTo(`${process.env.APP_URL || 'http://localhost:9001/'}signin`); +export const createRegularUser = () => { + return new Role(resolveUrl('/signin'), async() => { + await t.eval(() => window.localStorage.clear()); - await t - .typeText(AngularJSSelector.byModel('ui_signup.email'), 'a@a') - .typeText(AngularJSSelector.byModel('ui_signup.password'), 'a') - .typeText(AngularJSSelector.byModel('ui_exclude.confirm'), 'a') - .typeText(AngularJSSelector.byModel('ui_signup.firstName'), 'John') - .typeText(AngularJSSelector.byModel('ui_signup.lastName'), 'Doe') - .typeText(AngularJSSelector.byModel('ui_signup.company'), 'DevNull LTD') - .click('#countryInput') - .click(Selector('span').withText('Brazil')) - .click('#signup_submit'); + // Disable "Getting started" modal. + await t.eval(() => window.localStorage.showGettingStarted = 'false'); - // close modal window - await t.click('.modal-header button.close'); + const page = new PageSignIn(); + await page.open(); + await page.login('a@a', 'a'); + }); }; - - -const igniteSignIn = async(t) => { - await t.navigateTo(`${process.env.APP_URL || 'http://localhost:9001/'}signin`); - - await t - .typeText(AngularJSSelector.byModel('ui.email'), 'a@a') - .typeText(AngularJSSelector.byModel('ui.password'), 'a') - .click('#signin_submit'); - - // close modal window - await t.click('.modal-header button.close'); -}; - -const signIn = process.env.IGNITE_MODULES ? require(path.join(process.env.IGNITE_MODULES, 'e2e/testcafe/roles.js')).igniteSignIn : igniteSignIn; -const signUp = process.env.IGNITE_MODULES ? require(path.join(process.env.IGNITE_MODULES, 'e2e/testcafe/roles.js')).igniteSignUp : igniteSignUp; - -module.exports = { signUp, signIn }; diff --git a/modules/web-console/e2e/testcafe/testcafe.js b/modules/web-console/e2e/testcafe/testcafe.js index 66610e8b6a4a3..ef44b53fd4536 100644 --- a/modules/web-console/e2e/testcafe/testcafe.js +++ b/modules/web-console/e2e/testcafe/testcafe.js @@ -23,7 +23,7 @@ require('app-module-path').addPath(__dirname); const argv = require('minimist')(process.argv.slice(2)); const envEnabled = argv.env; -const { startEnv, removeData } = require('./envtools'); +const { startEnv, dropTestDB } = require('./envtools'); const createTestCafe = require('testcafe'); @@ -52,7 +52,7 @@ createTestCafe('localhost', 1337, 1338) if (envEnabled) await startEnv(); - await removeData(); + await dropTestDB(); testcafe = tc; @@ -65,7 +65,7 @@ createTestCafe('localhost', 1337, 1338) .src(resolveFixturesPaths()) .browsers(BROWSERS) .reporter(reporter) - .run({ skipJsErrors: true }); + .run({ skipJsErrors: true, quarantineMode: true }); } catch (err) { console.log(err); @@ -78,7 +78,7 @@ createTestCafe('localhost', 1337, 1338) testcafe.close(); if (envEnabled) - await removeData(); + await dropTestDB(); console.log('Tests failed: ' + failedCount); From 3579db951abec81b7201ad91326ddc73ee828387 Mon Sep 17 00:00:00 2001 From: Vasiliy Sisko Date: Fri, 26 Jan 2018 10:22:50 +0700 Subject: [PATCH 052/314] IGNITE-7392 Visor Cmd: Added missing word 'factory'. (cherry picked from commit c0b14e5) --- .../ignite/visor/commands/cache/VisorCacheCommand.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 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 3571efbaafa04..582feff06e4dd 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 @@ -875,13 +875,13 @@ object VisorCacheCommand { cacheT += ("Rebalance Cache Order", rebalanceCfg.getRebalanceOrder) cacheT += ("Eviction Policy Enabled", bool2Str(evictCfg.getPolicy != null)) - cacheT += ("Eviction Policy", safe(evictCfg.getPolicy)) + cacheT += ("Eviction Policy Factory", safe(evictCfg.getPolicy)) cacheT += ("Eviction Policy Max Size", safe(evictCfg.getPolicyMaxSize)) cacheT += ("Eviction Filter", safe(evictCfg.getFilter)) cacheT += ("Near Cache Enabled", bool2Str(nearCfg.isNearEnabled)) cacheT += ("Near Start Size", nearCfg.getNearStartSize) - cacheT += ("Near Eviction Policy", safe(nearCfg.getNearEvictPolicy)) + cacheT += ("Near Eviction Policy Factory", safe(nearCfg.getNearEvictPolicy)) cacheT += ("Near Eviction Policy Max Size", safe(nearCfg.getNearEvictMaxSize)) cacheT += ("Default Lock Timeout", cfg.getDefaultLockTimeout) From c125fa2068b62ab5286888ab2a16b5b034964dfd Mon Sep 17 00:00:00 2001 From: Vasiliy Sisko Date: Mon, 12 Feb 2018 10:13:06 +0700 Subject: [PATCH 053/314] IGNITE-2093 Visor CMD: Added support to scan near cache. (cherry picked from commit a5a907b) --- .../commands/cache/VisorCacheCommand.scala | 29 +++++++++++++++---- .../cache/VisorCacheScanCommand.scala | 19 +++++++----- .../scala/org/apache/ignite/visor/visor.scala | 2 +- 3 files changed, 37 insertions(+), 13 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 582feff06e4dd..d55fed1e72402 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 @@ -71,7 +71,7 @@ import scala.language.{implicitConversions, reflectiveCalls} * cache -i {-system} * cache {-c=} {-id=|id8=} {-s=hi|mi|rd|wr|cn} {-a} {-r} {-system} * cache -clear {-c=} - * cache -scan -c= {-id=|id8=} {-p=} {-system} + * cache -scan -c= {-near} {-id=|id8=} {-p=} {-system} * cache -stop -c= * cache -reset -c= * cache -rebalance -c= @@ -112,9 +112,11 @@ import scala.language.{implicitConversions, reflectiveCalls} * -system * Enable showing of information about system caches. * -clear - * Clears cache. + * Clears cache. * -scan - * Prints list of all entries from cache. + * Prints list of all entries from cache. + * -near + * Prints list of all entries from near cache of cache. * -stop * Stop cache with specified name. * -reset @@ -153,6 +155,8 @@ import scala.language.{implicitConversions, reflectiveCalls} * with page of 50 items from all nodes with this cache. * cache -scan -c=cache -id8=12345678 * Prints list entries from cache with name 'cache' and node '12345678' ID8. + * cache -scan -c=cache -near -id8=12345678 + * Prints list entries from near cache of cache with name 'cache' and node '12345678' ID8. * cache -stop -c=cache * Stops cache with name 'cache'. * cache -reset -c=cache @@ -196,6 +200,9 @@ class VisorCacheCommand extends VisorConsoleCommand { * cache -scan -c=cache -id8=12345678 * Prints list entries from cache with name 'cache' and node '12345678' ID8. *
+ * cache -scan -c=cache -near -id8=12345678 + * Prints list entries from near cache of cache with name 'cache' and node '12345678' ID8. + *
* cache -stop -c=@c0 * Stop cache with name taken from 'c0' memory variable. *
@@ -227,7 +234,16 @@ class VisorCacheCommand extends VisorConsoleCommand { return - case Right(n) => n + case Right(n) => n match { + case None if hasArgName("scan", argLst) && hasArgName("near", argLst) => + askForNode("Select node from:") match { + case None => return + + case nidOpt => nidOpt.map(ignite.cluster.node(_)) + } + + case _ => n + } } val showSystem = hasArgFlag("system", argLst) @@ -707,7 +723,7 @@ object VisorCacheCommand { "cache -i", "cache {-c=} {-id=|id8=} {-s=hi|mi|rd|wr} {-a} {-r}", "cache -clear {-c=} {-id=|id8=}", - "cache -scan -c= {-id=|id8=} {-p=}", + "cache -scan -c= {-near} {-id=|id8=} {-p=}", "cache -stop -c=", "cache -reset -c=", "cache -rebalance -c=" @@ -733,6 +749,7 @@ object VisorCacheCommand { "-clear" -> "Clears cache.", "-system" -> "Enable showing of information about system caches.", "-scan" -> "Prints list of all entries from cache.", + "-near" -> "Prints list of all entries from near cache of cache.", "-stop" -> "Stop cache with specified name.", "-reset" -> "Reset metrics of cache with specified name.", "-rebalance" -> "Re-balance partitions for cache with specified name.", @@ -791,6 +808,8 @@ object VisorCacheCommand { "cache -scan -c=@c0 -p=50" -> ("Prints list entries from cache with name taken from 'c0' memory variable" + " with page of 50 items from all nodes with this cache."), "cache -scan -c=cache -id8=12345678" -> "Prints list entries from cache with name 'cache' and node '12345678' ID8.", + "cache -scan -near -c=cache -id8=12345678" -> + "Prints list entries from near cache of cache with name 'cache' and node '12345678' ID8.", "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.", "cache -rebalance -c=cache" -> "Re-balance partitions for cache with name 'cache'." diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheScanCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheScanCommand.scala index 8e46ffcf4c42f..72d4cabf5c8e4 100644 --- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheScanCommand.scala +++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheScanCommand.scala @@ -32,19 +32,21 @@ import scala.collection.JavaConversions._ * * ====Specification==== * {{{ - * cache {-id=|-id8=} {-p=} -c= -scan + * cache -scan -c= {-near} {-id=|-id8=} {-p=} * }}} * * ====Arguments==== * {{{ + * + * Name of the cache. + * + * Prints list of all entries from near cache of cache. * * Full node ID. * * Node ID8. * * Number of object to fetch from cache at once. - * - * Name of the cache. * }}} * * ====Examples==== @@ -54,8 +56,10 @@ import scala.collection.JavaConversions._ * cache -c=@c0 -scan -p=50 * List entries from cache with name taken from 'c0' memory variable with page of 50 items * from all nodes with this cache. - * cache -c=cache -scan -id8=12345678 + * cache -scan -c=cache -id8=12345678 * List entries from cache with name 'cache' and node '12345678' ID8. + * cache -scan -near -c=cache -id8=12345678 + * List entries from near cache of cache with name 'cache' and node '12345678' ID8. * }}} */ class VisorCacheScanCommand { @@ -100,6 +104,7 @@ class VisorCacheScanCommand { def scan(argLst: ArgList, node: Option[ClusterNode]) { val pageArg = argValue("p", argLst) val cacheArg = argValue("c", argLst) + val near = hasArgName("near", argLst) var pageSize = 25 @@ -138,7 +143,7 @@ class VisorCacheScanCommand { val firstPage = try executeRandom(groupForDataNode(node, cacheName), - classOf[VisorScanQueryTask], new VisorScanQueryTaskArg(cacheName, null, false, false, false, false, pageSize)) match { + classOf[VisorScanQueryTask], new VisorScanQueryTaskArg(cacheName, null, false, false, near, false, pageSize)) match { case x if x.getError != null => error(x.getError) @@ -157,7 +162,7 @@ class VisorCacheScanCommand { } if (firstPage.getRows.isEmpty) { - println(s"Cache: ${escapeName(cacheName)} is empty") + println(s"${if (near) "Near cache" else "Cache"}: ${escapeName(cacheName)} is empty") return } @@ -165,7 +170,7 @@ class VisorCacheScanCommand { var nextPage: VisorQueryResult = firstPage def render() { - println("Entries in cache: " + escapeName(cacheName)) + println(s"Entries in ${if (near) "near" else ""} cache: " + escapeName(cacheName)) val t = VisorTextTable() 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 c24bc25e3ca8a..795cc5322370e 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 @@ -1309,7 +1309,7 @@ object visor extends VisorTag { * * @return `True` when cluster is active. */ - def isActive: Boolean = ignite.active + def isActive: Boolean = ignite.cluster().active() /** * Gets timestamp of Visor console connection. Returns `0` if Visor console is not connected. From 2f074c05492439e1860a6920fff2861a071b58a8 Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Wed, 28 Feb 2018 18:53:26 +0700 Subject: [PATCH 054/314] IGNITE-7578 Backport for ignite-2.4.3. --- .../visor/node/VisorClientConnectorConfiguration.java | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorClientConnectorConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorClientConnectorConfiguration.java index 0a1459ee0d01e..9f2ca81d5c327 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorClientConnectorConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorClientConnectorConfiguration.java @@ -26,8 +26,6 @@ import org.apache.ignite.internal.visor.VisorDataTransferObject; import org.jetbrains.annotations.Nullable; -import static org.apache.ignite.internal.visor.util.VisorTaskUtils.compactClass; - /** * Data transfer object for client connector configuration. */ @@ -108,10 +106,10 @@ public VisorClientConnectorConfiguration(ClientConnectorConfiguration cfg) { jdbcEnabled = cfg.isJdbcEnabled(); odbcEnabled = cfg.isOdbcEnabled(); thinCliEnabled = cfg.isThinClientEnabled(); - sslEnabled = cfg.isSslEnabled(); - useIgniteSslCtxFactory = cfg.isUseIgniteSslContextFactory(); - sslClientAuth = cfg.isSslClientAuth(); - sslCtxFactory = compactClass(cfg.getSslContextFactory()); + sslEnabled = false; + useIgniteSslCtxFactory = false; + sslClientAuth = false; + sslCtxFactory = null; } /** @@ -225,6 +223,7 @@ public String getSslContextFactory() { return sslCtxFactory; } + /** {@inheritDoc} */ @Override public byte getProtocolVersion() { return V2; } From 8b1e77bac2d4fe125463f8978ca905d34a45b82f Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Wed, 28 Feb 2018 18:57:53 +0700 Subject: [PATCH 055/314] ignite-2.4.3 Fixed versions list. --- modules/web-console/frontend/app/services/Version.service.js | 4 ---- 1 file changed, 4 deletions(-) diff --git a/modules/web-console/frontend/app/services/Version.service.js b/modules/web-console/frontend/app/services/Version.service.js index 88e403b7a0b79..9bd9c7dfc302d 100644 --- a/modules/web-console/frontend/app/services/Version.service.js +++ b/modules/web-console/frontend/app/services/Version.service.js @@ -76,10 +76,6 @@ export default class IgniteVersion { this.webConsole = '2.4.0'; this.supportedVersions = [ - { - label: 'Ignite 2.5', - ignite: '2.5.0' - }, { label: 'Ignite 2.4', ignite: '2.4.0' From bad7053586cccbbae4eb3d6dcba0d1ada5d2d2f2 Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Thu, 1 Mar 2018 16:18:17 +0700 Subject: [PATCH 056/314] IGNITE-6920 Fixed missed require. (cherry picked from commit fe89fc4) --- modules/web-console/backend/index.js | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/modules/web-console/backend/index.js b/modules/web-console/backend/index.js index 013de47ebc2bf..ad334bcf698ca 100644 --- a/modules/web-console/backend/index.js +++ b/modules/web-console/backend/index.js @@ -20,7 +20,9 @@ const fs = require('fs'); const path = require('path'); -require('app-module-path').addPath(path.join(__dirname, 'node_modules')); +const appPath = require('app-module-path'); +appPath.addPath(__dirname); +appPath.addPath(path.join(__dirname, 'node_modules')); const _ = require('lodash'); const getos = require('getos'); From e899d0a0fa8fb1692ddc7f74ee43dc9ca50f885d Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Thu, 1 Mar 2018 15:41:52 +0300 Subject: [PATCH 057/314] IGNITE-7717 Fixed missed updateTopologyVersion call in case of a merged exchange - Fixes #3536. Signed-off-by: Alexey Goncharuk --- .../dht/GridClientPartitionTopology.java | 10 ++++++-- .../dht/GridDhtPartitionTopologyImpl.java | 11 +++++++-- .../GridDhtPartitionsExchangeFuture.java | 24 +++++++++++++++++++ 3 files changed, 41 insertions(+), 4 deletions(-) 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 def00f3f9ae89..5c98ee846ff60 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 @@ -206,9 +206,15 @@ private String mapString(GridDhtPartitionMap map) { try { AffinityTopologyVersion exchTopVer = exchFut.initialVersion(); - assert exchTopVer.compareTo(topVer) > 0 : "Invalid topology version [grp=" + grpId + + // Update is correct if topology version is newer or in case of newer discovery caches. + boolean isCorrectUpdate = exchTopVer.compareTo(topVer) > 0 + || (exchTopVer.compareTo(topVer) == 0 && this.discoCache != null && discoCache.version().compareTo(this.discoCache.version()) > 0); + + assert isCorrectUpdate : "Invalid topology version [grp=" + grpId + ", topVer=" + topVer + - ", exchVer=" + exchTopVer + ']'; + ", exchVer=" + exchTopVer + + ", discoCacheVer=" + (this.discoCache != null ? this.discoCache.version() : "None") + + ", exchDiscoCacheVer=" + discoCache.version() + ']'; this.stopping = stopping; 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 020c3e7ad34bb..12340421dc5c6 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 @@ -63,6 +63,7 @@ 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.U; +import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_PART_DATA_LOST; @@ -235,7 +236,7 @@ private String mapString(GridDhtPartitionMap map) { /** {@inheritDoc} */ @Override public void updateTopologyVersion( GridDhtTopologyFuture exchFut, - DiscoCache discoCache, + @NotNull DiscoCache discoCache, long updSeq, boolean stopping ) throws IgniteInterruptedCheckedException { @@ -244,9 +245,15 @@ private String mapString(GridDhtPartitionMap map) { try { AffinityTopologyVersion exchTopVer = exchFut.initialVersion(); - assert exchTopVer.compareTo(readyTopVer) > 0 : "Invalid topology version [grp=" + grp.cacheOrGroupName() + + // Update is correct if topology version is newer or in case of newer discovery caches. + boolean isCorrectUpdate = exchTopVer.compareTo(readyTopVer) > 0 + || (exchTopVer.compareTo(readyTopVer) == 0 && this.discoCache != null && discoCache.version().compareTo(this.discoCache.version()) > 0); + + assert isCorrectUpdate : "Invalid topology version [grp=" + grp.cacheOrGroupName() + ", topVer=" + readyTopVer + ", exchTopVer=" + exchTopVer + + ", discoCacheVer=" + (this.discoCache != null ? this.discoCache.version() : "None") + + ", exchDiscoCacheVer=" + discoCache.version() + ", fut=" + exchFut + ']'; this.stopping = stopping; 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 8172c7ab222d4..e87015c5e66e3 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 @@ -69,6 +69,7 @@ import org.apache.ignite.internal.processors.cache.ExchangeDiscoveryEvents; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate; +import org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.LocalJoinCachesContext; import org.apache.ignite.internal.processors.cache.StateChangeRequest; @@ -806,6 +807,8 @@ private void initTopologies() throws IgniteCheckedException { } /** + * Updates topology versions and discovery caches on all topologies. + * * @param crd Coordinator flag. * @throws IgniteCheckedException If failed. */ @@ -1838,6 +1841,18 @@ public boolean mergeJoinExchange(GridDhtPartitionsExchangeFuture fut) { return wait; } + /** + * Checks that some futures were merged to the current. + * Future without merges has only one DiscoveryEvent. + * If we merge futures to the current (see {@link GridCachePartitionExchangeManager#mergeExchanges(GridDhtPartitionsExchangeFuture, GridDhtPartitionsFullMessage)}) + * we add new discovery event from merged future. + * + * @return {@code True} If some futures were merged to current, false in other case. + */ + private boolean hasMergedExchanges() { + return context().events().events().size() > 1; + } + /** * @param fut Current future. * @return Pending join request if any. @@ -2382,6 +2397,12 @@ private void onAllReceived(@Nullable Collection sndResNodes) { boolean finish = cctx.exchange().mergeExchangesOnCoordinator(this); + // Synchronize in case of changed coordinator (thread switched to sys-*) + synchronized (mux) { + if (hasMergedExchanges()) + updateTopologies(true); + } + if (!finish) return; } @@ -2978,6 +2999,9 @@ private void processFullMessage(boolean checkCrd, ClusterNode node, GridDhtParti return; // Node is stopping, no need to further process exchange. } + if (hasMergedExchanges()) + updateTopologies(false); + assert resTopVer.equals(exchCtx.events().topologyVersion()) : "Unexpected result version [" + "msgVer=" + resTopVer + ", locVer=" + exchCtx.events().topologyVersion() + ']'; From 1948dc73f36ddfda6bc55b9c1ea7421fb8c62ff1 Mon Sep 17 00:00:00 2001 From: Sergey Kosarev Date: Mon, 12 Feb 2018 10:51:57 +0700 Subject: [PATCH 058/314] IGNITE-7485 Added support for authentication parameters in control.sh. (cherry picked from commit f53c48a) --- .../internal/client/impl/GridClientImpl.java | 28 +-- .../GridClientConnectionManagerAdapter.java | 11 +- .../GridClientNioTcpConnection.java | 8 +- .../client/router/impl/GridTcpRouterImpl.java | 25 +- .../internal/commandline/Arguments.java | 117 +++++++++ .../internal/commandline/CommandHandler.java | 232 +++++++++++------- .../CommandHandlerParsingTest.java | 118 +++++++++ .../testsuites/IgniteUtilSelfTestSuite.java | 2 + .../ignite/util/GridCommandHandlerTest.java | 35 ++- 9 files changed, 405 insertions(+), 171 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/commandline/Arguments.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/commandline/CommandHandlerParsingTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/GridClientImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/GridClientImpl.java index 9b71ae4d77cad..d045a62ff9371 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/GridClientImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/GridClientImpl.java @@ -17,8 +17,6 @@ package org.apache.ignite.internal.client.impl; -import java.lang.reflect.Constructor; -import java.lang.reflect.InvocationTargetException; import java.net.InetSocketAddress; import java.util.ArrayList; import java.util.Collection; @@ -40,6 +38,7 @@ import org.apache.ignite.internal.client.GridClient; import org.apache.ignite.internal.client.GridClientCacheMode; import org.apache.ignite.internal.client.GridClientClosedException; +import org.apache.ignite.internal.client.GridClientClusterState; import org.apache.ignite.internal.client.GridClientCompute; import org.apache.ignite.internal.client.GridClientConfiguration; import org.apache.ignite.internal.client.GridClientData; @@ -48,7 +47,6 @@ import org.apache.ignite.internal.client.GridClientDisconnectedException; import org.apache.ignite.internal.client.GridClientException; import org.apache.ignite.internal.client.GridClientFactory; -import org.apache.ignite.internal.client.GridClientClusterState; import org.apache.ignite.internal.client.GridClientNode; import org.apache.ignite.internal.client.GridClientPartitionAffinity; import org.apache.ignite.internal.client.GridClientPredicate; @@ -70,10 +68,6 @@ * Client implementation. */ public class GridClientImpl implements GridClient { - /** Enterprise connection manager class name. */ - private static final String ENT_CONN_MGR_CLS = - "org.apache.ignite.internal.client.impl.connection.GridClientConnectionManagerEntImpl"; - /** Null mask object. */ private static final Object NULL_MASK = new Object(); @@ -445,25 +439,7 @@ private GridClientConnectionManager createConnectionManager(UUID clientId, SSLCo GridClientConfiguration cfg, Collection routers, GridClientTopology top, @Nullable Byte marshId, boolean routerClient) throws GridClientException { - GridClientConnectionManager mgr; - - try { - Class cls = Class.forName(ENT_CONN_MGR_CLS); - - Constructor cons = cls.getConstructor(UUID.class, SSLContext.class, GridClientConfiguration.class, - Collection.class, GridClientTopology.class, Byte.class, boolean.class); - - mgr = (GridClientConnectionManager)cons.newInstance(clientId, sslCtx, cfg, routers, top, marshId, - routerClient); - } - catch (ClassNotFoundException ignored) { - mgr = new GridClientConnectionManagerOsImpl(clientId, sslCtx, cfg, routers, top, marshId, routerClient); - } - catch (NoSuchMethodException | IllegalAccessException | InstantiationException | InvocationTargetException e) { - throw new GridClientException("Failed to create client connection manager.", e); - } - - return mgr; + return new GridClientConnectionManagerOsImpl(clientId, sslCtx, cfg, routers, top, marshId, routerClient); } /** 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 aa0632229c6fd..87980b9c1da1d 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 @@ -472,7 +472,7 @@ protected GridClientConnection connect(@Nullable UUID nodeId, InetSocketAddress try { conn = new GridClientNioTcpConnection(srv, clientId, addr, sslCtx, pingExecutor, cfg.getConnectTimeout(), cfg.getPingInterval(), cfg.getPingTimeout(), - cfg.isTcpNoDelay(), marsh, marshId, top, cred, keepBinariesThreadLocal()); + cfg.isTcpNoDelay(), marsh, marshId, top, cred); } catch (GridClientException e) { if (marsh instanceof GridClientZipOptimizedMarshaller) { @@ -482,7 +482,7 @@ protected GridClientConnection connect(@Nullable UUID nodeId, InetSocketAddress conn = new GridClientNioTcpConnection(srv, clientId, addr, sslCtx, pingExecutor, cfg.getConnectTimeout(), cfg.getPingInterval(), cfg.getPingTimeout(), cfg.isTcpNoDelay(), ((GridClientZipOptimizedMarshaller)marsh).defaultMarshaller(), marshId, - top, cred, keepBinariesThreadLocal()); + top, cred); } else throw e; @@ -506,13 +506,6 @@ protected GridClientConnection connect(@Nullable UUID nodeId, InetSocketAddress } } - /** - * @return Get thread local used to enable keep binary mode. - */ - protected ThreadLocal keepBinariesThreadLocal() { - return null; - } - /** {@inheritDoc} */ @Override public void terminateConnection(GridClientConnection conn, GridClientNode node, Throwable e) { if (log.isLoggable(Level.FINE)) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java index 3bedd5fbc6d29..e117cc88391b9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java @@ -158,9 +158,6 @@ public class GridClientNioTcpConnection extends GridClientConnection { /** Marshaller. */ private final GridClientMarshaller marsh; - /** */ - private final ThreadLocal keepBinariesMode; - /** * Creates a client facade, tries to connect to remote server, in case of success starts reader thread. * @@ -192,8 +189,7 @@ public class GridClientNioTcpConnection extends GridClientConnection { GridClientMarshaller marsh, Byte marshId, GridClientTopology top, - Object cred, - ThreadLocal keepBinariesMode + Object cred ) throws IOException, GridClientException { super(clientId, srvAddr, sslCtx, top, cred); @@ -202,7 +198,6 @@ public class GridClientNioTcpConnection extends GridClientConnection { this.marsh = marsh; this.pingInterval = pingInterval; this.pingTimeout = pingTimeout; - this.keepBinariesMode = keepBinariesMode; SocketChannel ch = null; Socket sock = null; @@ -1062,7 +1057,6 @@ private static class TcpClientFuture extends GridClientFutureAdapter { private GridClientMessage pendingMsg; /** Flag indicating whether authentication retry was attempted for this request. */ - @SuppressWarnings("RedundantFieldInitialization") private int authRetry = STATE_INITIAL; /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterImpl.java index da55ec72da9a5..d6daf9994ac58 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterImpl.java @@ -18,8 +18,6 @@ package org.apache.ignite.internal.client.router.impl; import java.lang.management.ManagementFactory; -import java.lang.reflect.Constructor; -import java.lang.reflect.InvocationTargetException; import java.net.InetAddress; import java.net.UnknownHostException; import java.nio.ByteOrder; @@ -53,9 +51,6 @@ * Wrapper class for router process. */ public class GridTcpRouterImpl implements GridTcpRouter, GridTcpRouterMBean, LifecycleAware { - /** */ - private static final String ENT_NIO_LSNR_CLS = "org.apache.ignite.client.router.impl.GridTcpRouterNioListenerEntImpl"; - /** Id. */ private final UUID id = UUID.randomUUID(); @@ -108,23 +103,7 @@ public GridTcpRouterImpl(GridTcpRouterConfiguration cfg) { throw new IgniteException("Failed to initialise embedded client.", e); } - GridNioServerListener lsnr; - - try { - Class cls = Class.forName(ENT_NIO_LSNR_CLS); - - Constructor cons = cls.getDeclaredConstructor(IgniteLogger.class, GridRouterClientImpl.class); - - cons.setAccessible(true); - - lsnr = (GridNioServerListener)cons.newInstance(log, client); - } - catch (ClassNotFoundException ignored) { - lsnr = new GridTcpRouterNioListenerOsImpl(log, client); - } - catch (NoSuchMethodException | IllegalAccessException | InstantiationException | InvocationTargetException e) { - throw new IgniteException("Failed to create NIO listener.", e); - } + GridNioServerListener lsnr = new GridTcpRouterNioListenerOsImpl(log, client); parser = new GridTcpRouterNioParser(); @@ -379,4 +358,4 @@ private GridRouterClientImpl createClient(GridTcpRouterConfiguration routerCfg) return id.equals(that.id); } -} \ No newline at end of file +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/Arguments.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/Arguments.java new file mode 100644 index 0000000000000..6a9a3f7275e03 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/Arguments.java @@ -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. + */ + +package org.apache.ignite.internal.commandline; + +/** + * Bean with all parsed and validated arguments. + */ +public class Arguments { + /** Command. */ + private String cmd; + + /** Host. */ + private String host; + + /** Port. */ + private String port; + + /** User. */ + private String user; + + /** Password. */ + private String pwd; + + /** + * Action for baseline command. + */ + private String baselineAct; + + /** + * Arguments for baseline command. + */ + private String baselineArgs; + + /** + * @param cmd Command. + * @param host Host. + * @param port Port. + * @param user User. + * @param pwd Password. + * @param baselineAct Baseline action. + * @param baselineArgs Baseline args. + */ + public Arguments(String cmd, String host, String port, String user, String pwd, String baselineAct, + String baselineArgs) { + this.cmd = cmd; + this.host = host; + this.port = port; + this.user = user; + this.pwd = pwd; + this.baselineAct = baselineAct; + this.baselineArgs = baselineArgs; + } + + /** + * @return command + */ + public String command() { + return cmd; + } + + /** + * @return host name + */ + public String host() { + return host; + } + + /** + * @return port number + */ + public String port() { + return port; + } + + /** + * @return user name + */ + public String user() { + return user; + } + + /** + * @return password + */ + public String password() { + return pwd; + } + + /** + * @return baseline action + */ + public String baselineAction() { + return baselineAct; + } + + /** + * @return baseline arguments + */ + public String baselineArguments() { + return baselineArgs; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java index cf331f86db46b..5895ad2001b9a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java @@ -44,6 +44,9 @@ import org.apache.ignite.internal.visor.baseline.VisorBaselineTask; import org.apache.ignite.internal.visor.baseline.VisorBaselineTaskArg; import org.apache.ignite.internal.visor.baseline.VisorBaselineTaskResult; +import org.apache.ignite.plugin.security.SecurityCredentials; +import org.apache.ignite.plugin.security.SecurityCredentialsBasicProvider; +import org.jetbrains.annotations.NotNull; import static org.apache.ignite.internal.IgniteVersionUtils.ACK_VER_STR; import static org.apache.ignite.internal.IgniteVersionUtils.COPYRIGHT; @@ -58,10 +61,10 @@ */ public class CommandHandler { /** */ - private static final String DFLT_HOST = "127.0.0.1"; + static final String DFLT_HOST = "127.0.0.1"; /** */ - private static final String DFLT_PORT = "11211"; + static final String DFLT_PORT = "11211"; /** */ private static final String CMD_HELP = "--help"; @@ -73,16 +76,10 @@ public class CommandHandler { private static final String CMD_PORT = "--port"; /** */ - private static final String CMD_ACTIVATE = "--activate"; + private static final String CMD_PASSWORD = "--password"; /** */ - private static final String CMD_DEACTIVATE = "--deactivate"; - - /** */ - private static final String CMD_STATE = "--state"; - - /** */ - private static final String CMD_BASE_LINE = "--baseline"; + private static final String CMD_USER = "--user"; /** */ private static final String BASELINE_ADD = "add"; @@ -99,6 +96,18 @@ public class CommandHandler { /** */ private static final String DELIM = "--------------------------------------------------------------------------------"; + /** */ + static final String CMD_ACTIVATE = "--activate"; + + /** */ + static final String CMD_BASE_LINE = "--baseline"; + + /** */ + static final String CMD_DEACTIVATE = "--deactivate"; + + /** */ + static final String CMD_STATE = "--state"; + /** */ public static final int EXIT_CODE_OK = 0; @@ -157,18 +166,6 @@ private int error(int errCode, String s, Throwable e) { return errCode; } - /** - * Print command usage. - * - * @param desc Command description. - * @param cmd Command. - */ - private void usage(String desc, String cmd) { - log(desc); - log(" control.sh [--host HOST_OR_IP] [--port PORT] " + cmd); - nl(); - } - /** * Extract next argument. * @@ -189,6 +186,102 @@ private String nextArg(Iterator it, String err) { throw new IllegalArgumentException(err); } + /** + * Parses and validates arguments. + * + * @param rawArgs Array of arguments. + * @return Arguments bean. + * @throws IllegalArgumentException In case arguments aren't valid. + */ + @NotNull Arguments parseAndValidate(String... rawArgs) { + String host = DFLT_HOST; + + String port = DFLT_PORT; + + String user = null; + + String pwd = null; + + String baselineAct = ""; + + String baselineArgs = ""; + + List commands = new ArrayList<>(); + + Iterator it = Arrays.asList(rawArgs).iterator(); + + while (it.hasNext()) { + String str = it.next().toLowerCase(); + + switch (str) { + case CMD_HOST: + host = nextArg(it, "Expected host name"); + break; + + case CMD_PORT: + port = nextArg(it, "Expected port number"); + + try { + int p = Integer.parseInt(port); + + if (p <= 0 || p > 65535) + throw new IllegalArgumentException("Invalid value for port: " + port); + } + catch (NumberFormatException ignored) { + throw new IllegalArgumentException("Invalid value for port: " + port); + } + break; + + case CMD_USER: + user = nextArg(it, "Expected user name"); + break; + + case CMD_PASSWORD: + pwd = nextArg(it, "Expected password"); + break; + + case CMD_ACTIVATE: + case CMD_DEACTIVATE: + case CMD_STATE: + commands.add(str); + break; + + case CMD_BASE_LINE: + commands.add(CMD_BASE_LINE); + + if (it.hasNext()) { + baselineAct = it.next().toLowerCase(); + + if (BASELINE_ADD.equals(baselineAct) || BASELINE_REMOVE.equals(baselineAct) || + BASELINE_SET.equals(baselineAct) || BASELINE_SET_VERSION.equals(baselineAct)) + baselineArgs = nextArg(it, "Expected baseline arguments"); + else + throw new IllegalArgumentException("Unexpected argument for " + CMD_BASE_LINE + ": " + + baselineAct); + } + + } + } + + int sz = commands.size(); + + if (sz < 1) + throw new IllegalArgumentException("No action was specified"); + + if (sz > 1) + throw new IllegalArgumentException("Only one action can be specified, but found: " + sz); + + String cmd = commands.get(0); + + boolean hasUsr = F.isEmpty(user); + boolean hasPwd = F.isEmpty(pwd); + + if (hasUsr != hasPwd) + throw new IllegalArgumentException("Both user and password should be specified"); + + return new Arguments(cmd, host, port, user, pwd, baselineAct, baselineArgs); + } + /** * Activate cluster. * @@ -500,20 +593,32 @@ private boolean isConnectionError(Throwable e) { e instanceof GridServerUnreachableException; } + /** + * Print command usage. + * + * @param desc Command description. + * @param cmd Command. + */ + private void usage(String desc, String cmd) { + log(desc); + log(" control.sh [--host HOST_OR_IP] [--port PORT] [--user USER] [--password PASSWORD] " + cmd); + nl(); + } + /** * Parse and execute command. * - * @param args Arguments to parse and execute. + * @param rawArgs Arguments to parse and execute. * @return Exit code. */ - public int execute(String... args) { + public int execute(String... rawArgs) { log("Control utility [ver. " + ACK_VER_STR + "]"); log(COPYRIGHT); log("User: " + System.getProperty("user.name")); log(DELIM); try { - if (F.isEmpty(args) || (args.length == 1 && CMD_HELP.equalsIgnoreCase(args[0]))){ + if (F.isEmpty(rawArgs) || (rawArgs.length == 1 && CMD_HELP.equalsIgnoreCase(rawArgs[0]))) { log("This utility can do the following commands:"); usage(" Activate cluster:", CMD_ACTIVATE); @@ -540,79 +645,20 @@ public int execute(String... args) { return EXIT_CODE_OK; } - String host = DFLT_HOST; - - String port = DFLT_PORT; - - String baselineAct = ""; - - String baselineArgs = ""; - - List commands = new ArrayList<>(); - - Iterator it = Arrays.asList(args).iterator(); - - while (it.hasNext()) { - String str = it.next().toLowerCase(); - - switch (str) { - case CMD_HOST: - host = nextArg(it, "Expected host name"); - break; - - case CMD_PORT: - port = nextArg(it, "Expected port number"); - - try { - int p = Integer.parseInt(port); - - if (p <= 0 || p > 65535) - throw new IllegalArgumentException("Invalid value for port: " + port); - } - catch (NumberFormatException ignored) { - throw new IllegalArgumentException("Invalid value for port: " + port); - } - break; + Arguments args = parseAndValidate(rawArgs); - case CMD_ACTIVATE: - case CMD_DEACTIVATE: - case CMD_STATE: - commands.add(str); - break; - - case CMD_BASE_LINE: - commands.add(CMD_BASE_LINE); - - if (it.hasNext()) { - baselineAct = it.next().toLowerCase(); + GridClientConfiguration cfg = new GridClientConfiguration(); - if (BASELINE_ADD.equals(baselineAct) || BASELINE_REMOVE.equals(baselineAct) || - BASELINE_SET.equals(baselineAct) || BASELINE_SET_VERSION.equals(baselineAct)) - baselineArgs = nextArg(it, "Expected baseline arguments"); - else - throw new IllegalArgumentException("Unexpected argument for " + CMD_BASE_LINE + ": " - + baselineAct); - } + cfg.setServers(Collections.singletonList(args.host() + ":" + args.port())); - } + if (!F.isEmpty(args.user())) { + cfg.setSecurityCredentialsProvider( + new SecurityCredentialsBasicProvider(new SecurityCredentials(args.user(), args.password()))); } - int sz = commands.size(); - - if (sz < 1) - throw new IllegalArgumentException("No action was specified"); - - if (sz > 1) - throw new IllegalArgumentException("Only one action can be specified, but found: " + sz); - - GridClientConfiguration cfg = new GridClientConfiguration(); - - cfg.setServers(Collections.singletonList(host + ":" + port)); - try (GridClient client = GridClientFactory.start(cfg)) { - String cmd = commands.get(0); - switch (cmd) { + switch (args.command()) { case CMD_ACTIVATE: activate(client); break; @@ -626,7 +672,7 @@ public int execute(String... args) { break; case CMD_BASE_LINE: - baseline(client, baselineAct, baselineArgs); + baseline(client, args.baselineAction(), args.baselineArguments()); break; } } @@ -643,7 +689,6 @@ public int execute(String... args) { if (isConnectionError(e)) return error(EXIT_CODE_CONNECTION_FAILED, "Connection to cluster failed.", e); - return error(EXIT_CODE_UNEXPECTED_ERROR, "", e); } } @@ -657,3 +702,4 @@ public static void main(String[] args) { System.exit(hnd.execute(args)); } } + diff --git a/modules/core/src/test/java/org/apache/ignite/internal/commandline/CommandHandlerParsingTest.java b/modules/core/src/test/java/org/apache/ignite/internal/commandline/CommandHandlerParsingTest.java new file mode 100644 index 0000000000000..1ba8b3d20a243 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/commandline/CommandHandlerParsingTest.java @@ -0,0 +1,118 @@ +/* + * + * * Licensed to the Apache Software Foundation (ASF) under one or more + * * contributor license agreements. See the NOTICE file distributed with + * * this work for additional information regarding copyright ownership. + * * The ASF licenses this file to You under the Apache License, Version 2.0 + * * (the "License"); you may not use this file except in compliance with + * * the License. You may obtain a copy of the License at + * * + * * http://www.apache.org/licenses/LICENSE-2.0 + * * + * * Unless required by applicable law or agreed to in writing, software + * * distributed under the License is distributed on an "AS IS" BASIS, + * * WITHOUT WARRANTIES OR CONDITIONS OF 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.commandline; + +import junit.framework.TestCase; + +import static org.apache.ignite.internal.commandline.CommandHandler.CMD_ACTIVATE; +import static org.apache.ignite.internal.commandline.CommandHandler.CMD_BASE_LINE; +import static org.apache.ignite.internal.commandline.CommandHandler.CMD_DEACTIVATE; +import static org.apache.ignite.internal.commandline.CommandHandler.CMD_STATE; +import static org.apache.ignite.internal.commandline.CommandHandler.DFLT_HOST; +import static org.apache.ignite.internal.commandline.CommandHandler.DFLT_PORT; + +/** + * Tests Command Handler parsing arguments. + */ +public class CommandHandlerParsingTest extends TestCase { + /** Commands to test. */ + private static final String[] Commands = new String[] {CMD_STATE, CMD_ACTIVATE, CMD_DEACTIVATE, CMD_BASE_LINE}; + + /** + * Test parsing and validation for user and password arguments + */ + public void testParseAndValidateUserAndPassword() { + CommandHandler hnd = new CommandHandler(); + + for (String cmd : Commands) { + try { + hnd.parseAndValidate("--user"); + + fail("expected exception: Expected user name"); + } + catch (IllegalArgumentException e) { + e.printStackTrace(); + } + + try { + hnd.parseAndValidate("--password"); + + fail("expected exception: Expected password"); + } + catch (IllegalArgumentException e) { + e.printStackTrace(); + } + + try { + hnd.parseAndValidate("--user", "testUser", cmd); + + fail("expected exception: Both user and password should be specified"); + } + catch (IllegalArgumentException e) { + e.printStackTrace(); + } + + try { + hnd.parseAndValidate("--password", "testPass", cmd); + + fail("expected exception: Both user and password should be specified"); + } + catch (IllegalArgumentException e) { + e.printStackTrace(); + } + + Arguments args = hnd.parseAndValidate("--user", "testUser", "--password", "testPass", cmd); + + assertEquals("testUser", args.user()); + assertEquals("testPass", args.password()); + assertEquals(cmd, args.command()); + } + } + + /** + * tests host and port arguments + */ + public void testHostAndPort() { + CommandHandler hnd = new CommandHandler(); + + for (String cmd : Commands) { + Arguments args = hnd.parseAndValidate(cmd); + + assertEquals(cmd, args.command()); + assertEquals(DFLT_HOST, args.host()); + assertEquals(DFLT_PORT, args.port()); + + args = hnd.parseAndValidate("--port", "12345", "--host", "test-host", cmd); + + assertEquals(cmd, args.command()); + assertEquals("test-host", args.host()); + assertEquals("12345", args.port()); + + try { + hnd.parseAndValidate("--port", "wrong-port", cmd); + + fail("expected exception: Invalid value for port:"); + } + catch (IllegalArgumentException e) { + e.printStackTrace(); + } + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java index e524d046ecddc..e3be1e3f22312 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java @@ -19,6 +19,7 @@ import java.util.Set; import junit.framework.TestSuite; +import org.apache.ignite.internal.commandline.CommandHandlerParsingTest; import org.apache.ignite.internal.pagemem.impl.PageIdUtilsSelfTest; import org.apache.ignite.internal.processors.cache.GridCacheUtilsSelfTest; import org.apache.ignite.internal.util.GridArraysSelfTest; @@ -113,6 +114,7 @@ public static TestSuite suite(Set ignoredTests) throws Exception { suite.addTestSuite(PageIdUtilsSelfTest.class); // control.sh + suite.addTestSuite(CommandHandlerParsingTest.class); suite.addTestSuite(GridCommandHandlerTest.class); return suite; diff --git a/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java b/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java index c7850c6b4ebc0..0be131b4e5dac 100644 --- a/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java +++ b/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java @@ -89,11 +89,20 @@ public void testActivate() throws Exception { CommandHandler cmd = new CommandHandler(); - assertEquals(EXIT_CODE_OK, cmd.execute("--activate")); + assertEquals(EXIT_CODE_OK, execute(cmd, "--activate")); assertTrue(ignite.active()); } + /** + * @param cmd CommandHandler + * @param args arguments + * @return result of execution + */ + protected int execute(CommandHandler cmd, String... args) { + return cmd.execute(args); + } + /** * Test deactivation works via control.sh * @@ -110,7 +119,7 @@ public void testDeactivate() throws Exception { CommandHandler cmd = new CommandHandler(); - assertEquals(EXIT_CODE_OK, cmd.execute("--deactivate")); + assertEquals(EXIT_CODE_OK, execute(cmd, "--deactivate")); assertFalse(ignite.active()); } @@ -127,11 +136,11 @@ public void testState() throws Exception { CommandHandler cmd = new CommandHandler(); - assertEquals(EXIT_CODE_OK, cmd.execute("--state")); + assertEquals(EXIT_CODE_OK, execute(cmd, "--state")); ignite.active(true); - assertEquals(EXIT_CODE_OK, cmd.execute("--state")); + assertEquals(EXIT_CODE_OK, execute(cmd, "--state")); } /** @@ -148,7 +157,7 @@ public void testBaselineCollect() throws Exception { CommandHandler cmd = new CommandHandler(); - cmd.execute("--baseline"); + assertEquals(EXIT_CODE_OK, execute(cmd, "--baseline")); assertEquals(1, ignite.cluster().currentBaselineTopology().size()); } @@ -188,8 +197,8 @@ public void testBaselineAdd() throws Exception { Ignite other = startGrid(2); - assertEquals(EXIT_CODE_OK, cmd.execute("--baseline", "add", consistentIds(other))); - assertEquals(EXIT_CODE_OK, cmd.execute("--baseline", "add", consistentIds(other))); + assertEquals(EXIT_CODE_OK, execute(cmd, "--baseline", "add", consistentIds(other))); + assertEquals(EXIT_CODE_OK, execute(cmd, "--baseline", "add", consistentIds(other))); assertEquals(2, ignite.cluster().currentBaselineTopology().size()); } @@ -213,8 +222,8 @@ public void testBaselineRemove() throws Exception { CommandHandler cmd = new CommandHandler(); - assertEquals(EXIT_CODE_OK, cmd.execute("--baseline")); - assertEquals(EXIT_CODE_OK, cmd.execute("--baseline", "remove", offlineNodeConsId)); + assertEquals(EXIT_CODE_OK, execute(cmd, "--baseline")); + assertEquals(EXIT_CODE_OK, execute(cmd, "--baseline", "remove", offlineNodeConsId)); assertEquals(1, ignite.cluster().currentBaselineTopology().size()); } @@ -235,11 +244,11 @@ public void testBaselineSet() throws Exception { CommandHandler cmd = new CommandHandler(); - assertEquals(EXIT_CODE_OK, cmd.execute("--baseline", "set", consistentIds(ignite, other))); + assertEquals(EXIT_CODE_OK, execute(cmd, "--baseline", "set", consistentIds(ignite, other))); assertEquals(2, ignite.cluster().currentBaselineTopology().size()); - assertEquals(EXIT_CODE_UNEXPECTED_ERROR, cmd.execute("--baseline", "set", "invalidConsistentId")); + assertEquals(EXIT_CODE_UNEXPECTED_ERROR, execute(cmd, "--baseline", "set", "invalidConsistentId")); } /** @@ -258,9 +267,9 @@ public void testBaselineVersion() throws Exception { startGrid(2); - assertEquals(EXIT_CODE_OK, cmd.execute("--baseline")); + assertEquals(EXIT_CODE_OK, execute(cmd, "--baseline")); - assertEquals(EXIT_CODE_OK, cmd.execute("--baseline", "version", String.valueOf(ignite.cluster().topologyVersion()))); + assertEquals(EXIT_CODE_OK, execute(cmd, "--baseline", "version", String.valueOf(ignite.cluster().topologyVersion()))); assertEquals(2, ignite.cluster().currentBaselineTopology().size()); } From 20611b28b89adde162ad380cb3a6d9bd4608eb8b Mon Sep 17 00:00:00 2001 From: Vasiliy Sisko Date: Thu, 1 Mar 2018 23:13:53 +0700 Subject: [PATCH 059/314] IGNITE-7462 Web Console: Actualized configuration generation. (cherry picked from commit 30fab74) --- modules/web-console/backend/app/schemas.js | 28 ++++++++++-- .../ui-ace-java/ui-ace-java.controller.js | 2 +- .../ui-ace-spring/ui-ace-spring.controller.js | 2 +- .../generator/AbstractTransformer.js | 8 ++-- .../generator/ConfigurationGenerator.js | 45 +++++++++++++------ .../generator/defaults/Cluster.service.js | 15 ++++++- .../configuration/clusters/checkpoint/s3.pug | 30 ++++++++++++- .../configuration/clusters/communication.pug | 2 +- .../clusters/general/discovery/s3.pug | 10 +++++ .../controllers/clusters-controller.js | 5 ++- 10 files changed, 118 insertions(+), 29 deletions(-) diff --git a/modules/web-console/backend/app/schemas.js b/modules/web-console/backend/app/schemas.js index b3d61ac44c8e6..553d2b272bf43 100644 --- a/modules/web-console/backend/app/schemas.js +++ b/modules/web-console/backend/app/schemas.js @@ -426,10 +426,13 @@ module.exports.factory = function(mongoose) { }, S3: { bucketName: String, + bucketEndpoint: String, + SSEAlgorithm: String, clientConfiguration: { protocol: {type: String, enum: ['HTTP', 'HTTPS']}, maxConnections: Number, - userAgent: String, + userAgentPrefix: String, + userAgentSuffix: String, localAddress: String, proxyHost: String, proxyPort: Number, @@ -470,7 +473,14 @@ module.exports.factory = function(mongoose) { useTcpKeepAlive: Boolean, dnsResolver: String, responseMetadataCacheSize: Number, - secureRandom: String + secureRandom: String, + cacheResponseMetadata: {type: Boolean, default: true}, + clientExecutionTimeout: Number, + nonProxyHosts: String, + socketSendBufferSizeHint: Number, + socketReceiveBufferSizeHint: Number, + useExpectContinue: {type: Boolean, default: true}, + useThrottleRetries: {type: Boolean, default: true} } }, Cloud: { @@ -784,10 +794,13 @@ module.exports.factory = function(mongoose) { } }, bucketNameSuffix: String, + bucketEndpoint: String, + SSEAlgorithm: String, clientConfiguration: { protocol: {type: String, enum: ['HTTP', 'HTTPS']}, maxConnections: Number, - userAgent: String, + userAgentPrefix: String, + userAgentSuffix: String, localAddress: String, proxyHost: String, proxyPort: Number, @@ -825,7 +838,14 @@ module.exports.factory = function(mongoose) { useTcpKeepAlive: Boolean, dnsResolver: String, responseMetadataCacheSize: Number, - secureRandom: String + secureRandom: String, + cacheResponseMetadata: {type: Boolean, default: true}, + clientExecutionTimeout: Number, + nonProxyHosts: String, + socketSendBufferSizeHint: Number, + socketReceiveBufferSizeHint: Number, + useExpectContinue: {type: Boolean, default: true}, + useThrottleRetries: {type: Boolean, default: true} }, checkpointListener: String }, 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 22f7d183d03de..efd317c552d32 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 @@ -98,7 +98,7 @@ export default ['IgniteVersion', 'JavaTransformer', function(Version, java) { return acc; }, []); - return java.clusterCheckpoint(cluster, clusterCaches); + return java.clusterCheckpoint(cluster, available, clusterCaches); }; break; 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 index 7eccf6daba3bd..5df734fa25aaf 100644 --- 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 @@ -94,7 +94,7 @@ export default ['IgniteVersion', 'SpringTransformer', function(Version, spring) return acc; }, []); - return spring.clusterCheckpoint(cluster, clusterCaches); + return spring.clusterCheckpoint(cluster, available, clusterCaches); }; break; 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 339dbea58d6df..e0aece76a810c 100644 --- a/modules/web-console/frontend/app/modules/configuration/generator/AbstractTransformer.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/AbstractTransformer.js @@ -89,8 +89,8 @@ export default class AbstractTransformer { } // Generate communication group. - static clusterCommunication(cluster) { - return this.toSection(this.generator.clusterCommunication(cluster)); + static clusterCommunication(cluster, available) { + return this.toSection(this.generator.clusterCommunication(cluster, available)); } // Generate REST access configuration. @@ -304,8 +304,8 @@ export default class AbstractTransformer { } // Generate caches configs. - static clusterCheckpoint(cluster, caches) { - return this.toSection(this.generator.clusterCheckpoint(cluster, caches)); + static clusterCheckpoint(cluster, available, caches) { + return this.toSection(this.generator.clusterCheckpoint(cluster, available, caches)); } // Generate domain model for general group. 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 645e3e34b05ee..fa47de698ee0b 100644 --- a/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js @@ -77,13 +77,13 @@ export default class IgniteConfigurationGenerator { this.clusterAtomics(cluster.atomicConfiguration, available, cfg); this.clusterBinary(cluster.binaryConfiguration, cfg); this.clusterCacheKeyConfiguration(cluster.cacheKeyConfiguration, cfg); - this.clusterCheckpoint(cluster, cluster.caches, cfg); + this.clusterCheckpoint(cluster, available, cluster.caches, cfg); if (available('2.3.0')) this.clusterClientConnector(cluster, available, cfg); this.clusterCollision(cluster.collision, cfg); - this.clusterCommunication(cluster, cfg); + this.clusterCommunication(cluster, available, cfg); this.clusterConnector(cluster.connector, cfg); // Since ignite 2.3 @@ -236,6 +236,11 @@ export default class IgniteConfigurationGenerator { ipFinder.stringProperty('bucketName'); + if (available('2.4.0')) { + ipFinder.stringProperty('bucketEndpoint') + .stringProperty('SSEAlgorithm'); + } + break; case 'Cloud': ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.cloud.TcpDiscoveryCloudIpFinder', @@ -355,7 +360,7 @@ export default class IgniteConfigurationGenerator { ipFinder.beanProperty('retryPolicy', retryPolicyBean); } - ipFinder.pathProperty('basePath', '/services') + ipFinder.pathProperty('basePath') .stringProperty('serviceName') .boolProperty('allowDuplicateRegistrations'); @@ -499,7 +504,7 @@ export default class IgniteConfigurationGenerator { } // Generate checkpoint configurations. - static clusterCheckpoint(cluster, caches, cfg = this.igniteConfigurationBean()) { + static clusterCheckpoint(cluster, available, caches, cfg = this.igniteConfigurationBean()) { const cfgs = _.filter(_.map(cluster.checkpointSpi, (spi) => { switch (_.get(spi, 'kind')) { case 'FS': @@ -585,12 +590,18 @@ export default class IgniteConfigurationGenerator { s3Bean.stringProperty('bucketNameSuffix'); + if (available('2.4.0')) { + s3Bean.stringProperty('bucketEndpoint') + .stringProperty('SSEAlgorithm'); + } + const clientBean = new Bean('com.amazonaws.ClientConfiguration', 'clientCfg', spi.S3.clientConfiguration, clusterDflts.checkpointSpi.S3.clientConfiguration); clientBean.enumProperty('protocol') .intProperty('maxConnections') - .stringProperty('userAgent'); + .stringProperty('userAgentPrefix') + .stringProperty('userAgentSuffix'); const locAddr = new Bean('java.net.InetAddress', '', spi.S3.clientConfiguration) .factoryMethod('getByName') @@ -609,7 +620,8 @@ export default class IgniteConfigurationGenerator { clientBean.property('proxyPassword', `checkpoint.s3.proxy.${userName}.password`); clientBean.stringProperty('proxyDomain') - .stringProperty('proxyWorkstation'); + .stringProperty('proxyWorkstation') + .stringProperty('nonProxyHosts'); const retryPolicy = spi.S3.clientConfiguration.retryPolicy; @@ -682,7 +694,8 @@ export default class IgniteConfigurationGenerator { break; case 'Custom': - retryBean = new Bean('com.amazonaws.retry.RetryPolicy', 'retryPolicy', policy); + retryBean = new Bean('com.amazonaws.retry.RetryPolicy', 'retryPolicy', policy, + clusterDflts.checkpointSpi.S3.clientConfiguration.retryPolicy); retryBean.beanConstructorArgument('retryCondition', retryBean.valueOf('retryCondition') ? new EmptyBean(retryBean.valueOf('retryCondition')) : null) .beanConstructorArgument('backoffStrategy', retryBean.valueOf('backoffStrategy') ? new EmptyBean(retryBean.valueOf('backoffStrategy')) : null) @@ -703,14 +716,17 @@ export default class IgniteConfigurationGenerator { .intProperty('socketTimeout') .intProperty('connectionTimeout') .intProperty('requestTimeout') - .intProperty('socketSendBufferSizeHints') .stringProperty('signerOverride') .intProperty('connectionTTL') .intProperty('connectionMaxIdleMillis') .emptyBeanProperty('dnsResolver') .intProperty('responseMetadataCacheSize') .emptyBeanProperty('secureRandom') + .intProperty('clientExecutionTimeout') .boolProperty('useReaper') + .boolProperty('cacheResponseMetadata') + .boolProperty('useExpectContinue') + .boolProperty('useThrottleRetries') .boolProperty('useGzip') .boolProperty('preemptiveBasicProxyAuth') .boolProperty('useTcpKeepAlive'); @@ -839,8 +855,8 @@ export default class IgniteConfigurationGenerator { colSpi.intProperty('parallelJobsNumber') .intProperty('waitingJobsNumber') - .intProperty('priorityAttributeKey') - .intProperty('jobPriorityAttributeKey') + .stringProperty('priorityAttributeKey') + .stringProperty('jobPriorityAttributeKey') .intProperty('defaultPriority') .intProperty('starvationIncrement') .boolProperty('starvationPreventionEnabled'); @@ -862,7 +878,7 @@ export default class IgniteConfigurationGenerator { } // Generate communication group. - static clusterCommunication(cluster, cfg = this.igniteConfigurationBean(cluster)) { + static clusterCommunication(cluster, available, cfg = this.igniteConfigurationBean(cluster)) { const commSpi = new Bean('org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi', 'communicationSpi', cluster.communication, clusterDflts.communication); @@ -893,8 +909,10 @@ export default class IgniteConfigurationGenerator { cfg.intProperty('networkTimeout') .intProperty('networkSendRetryDelay') - .intProperty('networkSendRetryCount') - .intProperty('discoveryStartupDelay'); + .intProperty('networkSendRetryCount'); + + if (available(['1.0.0', '2.3.0'])) + cfg.intProperty('discoveryStartupDelay'); return cfg; } @@ -1442,6 +1460,7 @@ export default class IgniteConfigurationGenerator { storageBean.stringProperty('storagePath') .intProperty('checkpointFrequency') .intProperty('checkpointThreads') + .enumProperty('checkpointWriteOrder') .enumProperty('walMode') .stringProperty('walPath') .stringProperty('walArchivePath') diff --git a/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.service.js b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.service.js index b55ed838eb979..8e8240313fb72 100644 --- a/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.service.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.service.js @@ -241,7 +241,8 @@ const DFLT_CLUSTER = { }, maxErrorRetry: { clsName: 'com.amazonaws.retry.PredefinedRetryPolicies' - } + }, + honorMaxErrorRetryInClientConfig: false }, maxErrorRetry: -1, socketTimeout: 50000, @@ -254,7 +255,13 @@ const DFLT_CLUSTER = { useReaper: true, useGzip: false, preemptiveBasicProxyAuth: false, - useTcpKeepAlive: false + useTcpKeepAlive: false, + cacheResponseMetadata: true, + clientExecutionTimeout: 0, + socketSendBufferSizeHint: 0, + socketReceiveBufferSizeHint: 0, + useExpectContinue: true, + useThrottleRetries: true } }, JDBC: { @@ -335,6 +342,10 @@ const DFLT_CLUSTER = { checkpointFrequency: 180000, checkpointPageBufferSize: 268435456, checkpointThreads: 4, + checkpointWriteOrder: { + clsName: 'org.apache.ignite.configuration.CheckpointWriteOrder', + value: 'SEQUENTIAL' + }, walMode: { clsName: 'org.apache.ignite.configuration.WALMode', value: 'DEFAULT' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/s3.pug b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/s3.pug index 16be6c0a818a6..d2f8c9e1bba21 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/s3.pug +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/s3.pug @@ -55,6 +55,16 @@ include /app/helpers/jade/mixins 'Custom AWS credentials provider implementation class', checkpointS3Custom) .settings-row +text('Bucket name suffix:', 'model.S3.bucketNameSuffix', '"checkpointS3BucketNameSuffix"', 'false', 'default-bucket', 'Bucket name suffix') +.pcb-flex-grid-break(ng-if-start=`$ctrl.available("2.4.0")`) +.settings-row + +text('Bucket endpoint:', `model.S3.bucketEndpoint`, '"checkpointS3BucketEndpoint"', false, 'Input bucket endpoint', + 'Bucket endpoint for IP finder
\ + For information about possible endpoint names visit
docs.aws.amazon.com') +.settings-row + +text('SSE algorithm:', `model.S3.SSEAlgorithm`, '"checkpointS3SseAlgorithm"', false, 'Input SSE algorithm', + 'Server-side encryption algorithm for Amazon S3-managed encryption keys
\ + For information about possible S3-managed encryption keys visit docs.aws.amazon.com') +.pcb-flex-grid-break(ng-if-end) .settings-row +java-class('Listener:', 'model.S3.checkpointListener', '"checkpointS3Listener" + $index', 'true', 'false', 'Checkpoint listener implementation class name', checkpointS3) @@ -75,8 +85,11 @@ include /app/helpers/jade/mixins +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') + +text('User agent prefix:', clientCfgModel + '.userAgentPrefix', '"checkpointS3UserAgentPrefix"', 'false', 'System specific header', + 'HTTP user agent prefix to send with all requests') + .details-row + +text('User agent suffix:', clientCfgModel + '.userAgentSuffix', '"checkpointS3UserAgentSuffix"', 'false', 'System specific header', + 'HTTP user agent suffix 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') @@ -95,6 +108,9 @@ include /app/helpers/jade/mixins .details-row +text('Proxy workstation:', clientCfgModel + '.proxyWorkstation', '"checkpointS3ProxyWorkstation"', 'false', 'Not specified', 'Optional Windows workstation name for configuring NTLM proxy support') + .details-row + +text('Non proxy hosts:', clientCfgModel + '.nonProxyHosts', '"checkpointS3NonProxyHosts"', 'false', 'Not specified', + 'Optional hosts the client will access without going through the proxy') .details-row +dropdown('Retry policy:', clientRetryModel + '.kind', '"checkpointS3RetryPolicy"', 'true', 'Default', '[\ {value: "Default", label: "Default SDK retry policy"},\ @@ -167,6 +183,16 @@ include /app/helpers/jade/mixins .details-row +java-class('SecureRandom class name:', clientCfgModel + '.secureRandom', '"checkpointS3SecureRandom" + $index', 'true', 'false', 'SecureRandom to be used by the SDK class name', checkpointS3) + .details-row + +number('Client execution timeout:', clientCfgModel + '.clientExecutionTimeout', '"checkpointS3ClientExecutionTimeout"', 'true', '0', '0', + 'Amount of time in milliseconds to allow the client to complete the execution of an API call
\ + 0 value disables that feature') + .details-row + +checkbox('Cache response metadata', clientCfgModel + '.cacheResponseMetadata', '"checkpointS3CacheResponseMetadata"', 'Cache response metadata') + .details-row + +checkbox('Use expect continue', clientCfgModel + '.useExpectContinue', '"checkpointS3UseExpectContinue"', 'Optional override to enable/disable support for HTTP/1.1 handshake utilizing EXPECT: 100-Continue') + .details-row + +checkbox('Use throttle retries', clientCfgModel + '.useThrottleRetries', '"checkpointS3UseThrottleRetries"', 'Retry throttling will be used') .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/communication.pug b/modules/web-console/frontend/app/modules/states/configuration/clusters/communication.pug index 93bb5cea34d7c..229472397d7a8 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/communication.pug +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/communication.pug @@ -38,7 +38,7 @@ include /app/helpers/jade/mixins +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') - .settings-row + .settings-row(ng-if='$ctrl.available(["1.0.0", "2.3.0"])') +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', 'Listener of communication events') diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/s3.pug b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/s3.pug index 81383fb7eff4f..90100b955fe41 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/s3.pug +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/s3.pug @@ -24,5 +24,15 @@ mixin discovery-s3(modelAt = 'backupItem') .details-row +text('Bucket name:', `${model}.bucketName`, `'${discoveryKind}BucketName'`, required, 'Input bucket name', 'Bucket name for IP finder') + .pcb-flex-grid-break(ng-if-start=`$ctrl.available("2.4.0")`) + .details-row + +text('Bucket endpoint:', `${model}.bucketEndpoint`, `'${discoveryKind}BucketEndpoint'`, false, 'Input bucket endpoint', + 'Bucket endpoint for IP finder
\ + For information about possible endpoint names visit docs.aws.amazon.com') + .details-row + +text('SSE algorithm:', `${model}.SSEAlgorithm`, `'${discoveryKind}SSEAlgorithm'`, false, 'Input SSE algorithm', + 'Server-side encryption algorithm for Amazon S3-managed encryption keys
\ + For information about possible S3-managed encryption keys visit docs.aws.amazon.com') + .pcb-flex-grid-break(ng-if-end) .details-row label Note, AWS credentials will be generated as stub \ No newline at end of file diff --git a/modules/web-console/frontend/controllers/clusters-controller.js b/modules/web-console/frontend/controllers/clusters-controller.js index 7d987dde3e00e..24d2c5438d6b7 100644 --- a/modules/web-console/frontend/controllers/clusters-controller.js +++ b/modules/web-console/frontend/controllers/clusters-controller.js @@ -164,7 +164,10 @@ export default ['$rootScope', '$scope', '$http', '$state', '$timeout', 'IgniteLe retryPolicy: { kind: 'Default' }, - useReaper: true + useReaper: true, + cacheResponseMetadata: true, + useExpectContinue: true, + useThrottleRetries: true } } }; From d4a07b6f195be963cc7f48bb4d63829e348948cf Mon Sep 17 00:00:00 2001 From: alexdel Date: Fri, 2 Mar 2018 09:30:29 +0700 Subject: [PATCH 060/314] IGNITE-7033 Web console: Adjusted width of columns on admin page. (cherry picked from commit 6733a4d) --- modules/web-console/frontend/app/app.js | 2 + .../list-of-registered-users/column-defs.js | 4 +- .../app/components/page-admin/index.js | 39 ++++++++++++ .../app/components/page-admin/style.scss | 62 +++++++++++++++++++ .../components/page-admin/template.tpl.pug} | 2 +- .../app/modules/states/admin.state.js | 15 +---- .../frontend/public/stylesheets/style.scss | 49 --------------- 7 files changed, 107 insertions(+), 66 deletions(-) create mode 100644 modules/web-console/frontend/app/components/page-admin/index.js create mode 100644 modules/web-console/frontend/app/components/page-admin/style.scss rename modules/web-console/frontend/{views/settings/admin.tpl.pug => app/components/page-admin/template.tpl.pug} (96%) diff --git a/modules/web-console/frontend/app/app.js b/modules/web-console/frontend/app/app.js index 6524450716e5c..bc1f7643367a7 100644 --- a/modules/web-console/frontend/app/app.js +++ b/modules/web-console/frontend/app/app.js @@ -116,6 +116,7 @@ import webConsoleFooter from './components/web-console-footer'; import igniteIcon from './components/ignite-icon'; import versionPicker from './components/version-picker'; import userNotifications from './components/user-notifications'; +import pageAdmin from './components/page-admin'; import pageConfigure from './components/page-configure'; import pageConfigureBasic from './components/page-configure-basic'; import pageConfigureAdvanced from './components/page-configure-advanced'; @@ -201,6 +202,7 @@ angular.module('ignite-console', [ igniteServices.name, versionPicker.name, userNotifications.name, + pageAdmin.name, pageConfigure.name, pageConfigureBasic.name, pageConfigureAdvanced.name, diff --git a/modules/web-console/frontend/app/components/list-of-registered-users/column-defs.js b/modules/web-console/frontend/app/components/list-of-registered-users/column-defs.js index 83382e0f4083c..e4ec91ef018f9 100644 --- a/modules/web-console/frontend/app/components/list-of-registered-users/column-defs.js +++ b/modules/web-console/frontend/app/components/list-of-registered-users/column-defs.js @@ -54,11 +54,11 @@ const VALUE_WITH_TITLE = '