Merge branch 'sprint-1' of
https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-sql-tests
Conflicts:
modules/core/src/main/java/org/apache/ignite/IgniteCache.java
modules/core/src/main/java/org/apache/ignite/cache/query/ContinuousQuery.java
modules/core/src/main/java/org/apache/ignite/cache/query/QueryAffinityPredicate.java
modules/core/src/main/java/org/apache/ignite/cache/query/SpiQuery.java
modules/core/src/main/java/org/apache/ignite/cache/query/SqlFieldsQuery.java
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java
modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryAck.java
Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/571223f6
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/571223f6
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/571223f6
Branch: refs/heads/ignite-sql-tests
Commit: 571223f6f74ac5a558231f830644506affaf99cf
Parents: 37877ee e23b86a
Author: S.Vladykin <[email protected]>
Authored: Mon Feb 9 18:18:46 2015 +0300
Committer: S.Vladykin <[email protected]>
Committed: Mon Feb 9 18:18:46 2015 +0300
----------------------------------------------------------------------
bin/ignite-schema-load.bat | 2 +-
bin/ignite-schema-load.sh | 2 +-
.../HibernateL2CacheExampleNodeStartup.java | 1 -
.../examples/ignitefs/IgniteFsExample.java | 6 +-
.../singlesplit/SingleSplitsLoadTest.java | 4 -
.../gridify/GridifySingleSplitLoadTest.java | 2 -
.../ClientAbstractMultiNodeSelfTest.java | 26 +-
.../internal/TaskEventSubjectIdSelfTest.java | 2 +-
.../JettyRestProcessorAbstractSelfTest.java | 2 +-
.../rest/RestProcessorMultiStartSelfTest.java | 2 +-
.../ignite/jdbc/JdbcComplexQuerySelfTest.java | 2 +-
.../ignite/jdbc/JdbcEmptyCacheSelfTest.java | 2 +-
.../ignite/jdbc/JdbcLocalCachesSelfTest.java | 2 +-
.../ignite/jdbc/JdbcMetadataSelfTest.java | 2 +-
.../jdbc/JdbcPreparedStatementSelfTest.java | 2 +-
.../ignite/jdbc/JdbcResultSetSelfTest.java | 2 +-
.../ignite/jdbc/JdbcStatementSelfTest.java | 2 +-
.../src/test/resources/spring-server-node.xml | 35 +-
.../test/resources/spring-server-ssl-node.xml | 35 +-
.../java/org/apache/ignite/IgniteCache.java | 13 +-
.../java/org/apache/ignite/IgniteCluster.java | 17 +-
.../org/apache/ignite/IgniteDataLoader.java | 2 +-
.../main/java/org/apache/ignite/IgniteFs.java | 6 +-
.../java/org/apache/ignite/IgniteLogger.java | 2 +-
.../java/org/apache/ignite/IgniteQueue.java | 10 +-
.../java/org/apache/ignite/IgniteServices.java | 13 +-
.../java/org/apache/ignite/IgniteStreamer.java | 5 +-
.../org/apache/ignite/IgniteTransactions.java | 3 +-
.../ignite/cache/query/ContinuousQuery.java | 3 +
.../client/impl/GridClientThreadFactory.java | 2 +-
.../ComputeExecutionRejectedException.java | 2 +-
.../ClientConnectionConfiguration.java | 66 +-
.../configuration/IgniteConfiguration.java | 407 +----
.../configuration/TransactionConfiguration.java | 3 +
.../ignite/ignitefs/IgniteFsFileSystem.java | 4 +-
.../apache/ignite/internal/GridDiagnostic.java | 2 +-
.../ignite/internal/GridJobContextImpl.java | 2 +-
.../ignite/internal/GridKernalContext.java | 44 +
.../ignite/internal/GridKernalContextImpl.java | 88 +-
.../apache/ignite/internal/GridProperties.java | 4 -
.../apache/ignite/internal/IgniteKernal.java | 54 +-
.../org/apache/ignite/internal/IgnitionEx.java | 275 +--
.../managers/communication/GridIoManager.java | 14 +-
.../cache/CacheStorePartialUpdateException.java | 3 +
.../cache/CacheWeakQueryIteratorsHolder.java | 3 +
.../processors/cache/GridCacheAdapter.java | 12 +
.../processors/cache/GridCacheMapEntry.java | 2 +-
.../processors/cache/IgniteCacheProxy.java | 16 +-
.../distributed/GridCacheTtlUpdateRequest.java | 3 +
.../cache/query/GridCacheSqlQuery.java | 3 +
.../cache/query/GridCacheTwoStepQuery.java | 3 +
.../cache/transactions/IgniteTxKey.java | 3 +
.../transactions/IgniteTxMetricsAdapter.java | 3 +
.../closure/GridClosureProcessor.java | 21 +-
.../processors/fs/GridGgfsDataManager.java | 2 +-
.../internal/processors/fs/GridGgfsImpl.java | 8 +-
.../hadoop/GridHadoopJobProperty.java | 2 +-
.../processors/job/GridJobProcessor.java | 2 +-
.../processors/query/GridQueryProcessor.java | 2 +-
.../processors/rest/GridRestProcessor.java | 2 +-
.../processors/task/GridTaskProcessor.java | 4 +-
.../ignite/internal/util/IgniteUtils.java | 26 +-
.../internal/util/future/IgniteFutureImpl.java | 3 +
.../util/portscanner/GridJmxPortFinder.java | 2 +-
.../node/VisorExecutorServiceConfiguration.java | 149 +-
.../optimized/optimized-classnames.properties | 1657 ++++++++++++++++++
.../jobstealing/JobStealingCollisionSpi.java | 16 +-
.../spi/collision/noop/NoopCollisionSpi.java | 2 +-
.../WeightedRandomLoadBalancingSpi.java | 4 +-
.../test/config/default-spring-url-testing.xml | 2 -
modules/core/src/test/config/example-cache.xml | 2 -
.../src/test/config/io-manager-benchmark.xml | 1 -
.../src/test/config/job-loadtest/client.xml | 41 +-
.../src/test/config/job-loadtest/server.xml | 15 +-
modules/core/src/test/config/jobs-load-base.xml | 28 +-
.../src/test/config/load/cache-benchmark.xml | 2 -
.../test/config/load/cache-client-benchmark.xml | 2 -
.../core/src/test/config/load/dsi-load-base.xml | 28 +-
.../src/test/config/load/merge-sort-base.xml | 41 +-
.../test/config/loaders/grid-cfg-2-grids.xml | 4 -
.../core/src/test/config/loaders/grid-cfg.xml | 5 -
.../config/spring-cache-put-remove-load.xml | 2 -
.../core/src/test/config/spring-multicache.xml | 39 +-
.../src/test/config/spring-start-nodes-attr.xml | 2 -
.../core/src/test/config/spring-start-nodes.xml | 2 -
.../config/streamer/spring-streamer-base.xml | 2 -
.../test/config/websession/spring-cache-1.xml | 2 -
.../test/config/websession/spring-cache-2.xml | 2 -
.../test/config/websession/spring-cache-3.xml | 2 -
...eJdbcStoreAbstractMultithreadedSelfTest.java | 9 +-
.../internal/GridCancelUnusedJobSelfTest.java | 9 +-
.../internal/GridDiscoveryEventSelfTest.java | 2 -
.../internal/GridManagementJobSelfTest.java | 168 --
.../ignite/internal/GridStartStopSelfTest.java | 6 -
.../cache/GridCacheAbstractFullApiSelfTest.java | 43 +-
.../GridCacheAbstractIteratorsSelfTest.java | 2 -
.../GridCacheDaemonNodeAbstractSelfTest.java | 2 -
.../cache/GridCacheDeploymentSelfTest.java | 2 -
.../cache/GridCacheLuceneQueryIndexTest.java | 1 -
...GridCacheQueueMultiNodeAbstractSelfTest.java | 19 +-
...dCacheMultithreadedFailoverAbstractTest.java | 1 -
.../GridCachePreloadLifecycleAbstractTest.java | 9 +-
...tomicClientOnlyMultiNodeFullApiSelfTest.java | 6 +-
...eAtomicNearOnlyMultiNodeFullApiSelfTest.java | 3 +-
.../GridCacheNearReaderPreloadSelfTest.java | 1 -
...achePartitionedMultiNodeFullApiSelfTest.java | 18 +-
.../local/GridCacheLocalIteratorsSelfTest.java | 80 -
.../dataload/GridDataLoaderPerformanceTest.java | 2 -
...heGgfsPerBlockLruEvictionPolicySelfTest.java | 2 -
.../processors/fs/GridGgfsAbstractSelfTest.java | 7 +-
.../fs/GridGgfsDualAbstractSelfTest.java | 2 +-
.../processors/fs/GridGgfsModesSelfTest.java | 138 +-
.../cache/GridCacheCommandHandlerSelfTest.java | 6 +-
.../GridServiceReassignmentSelfTest.java | 17 +-
...dStartupWithUndefinedIgniteHomeSelfTest.java | 1 -
.../util/future/GridFutureAdapterSelfTest.java | 8 +-
.../cache/GridCacheSingleNodeLoadTest.java | 6 +-
.../loadtests/colocation/spring-colocation.xml | 26 +-
.../multisplit/GridMultiSplitsLoadTest.java | 2 -
.../GridMultiSplitsRedeployLoadTest.java | 2 -
...ridSingleSplitsNewNodesAbstractLoadTest.java | 9 +-
.../GridSingleSplitsRedeployLoadTest.java | 2 -
...SessionCancelSiblingsFromFutureSelfTest.java | 9 +-
...ridSessionCancelSiblingsFromJobSelfTest.java | 9 +-
...idSessionCancelSiblingsFromTaskSelfTest.java | 9 +-
...GridSessionJobWaitTaskAttributeSelfTest.java | 10 +-
.../GridSessionSetFutureAttributeSelfTest.java | 9 +-
...nSetFutureAttributeWaitListenerSelfTest.java | 9 +-
.../GridSessionSetJobAttributeSelfTest.java | 10 +-
...sionSetJobAttributeWaitListenerSelfTest.java | 9 +-
.../GridSessionWaitAttributeSelfTest.java | 10 +-
.../spi/GridTcpSpiForwardingSelfTest.java | 1 -
.../spi/discovery/tcp/TcpDiscoverySelfTest.java | 2 -
.../tcp/TcpDiscoverySnapshotHistoryTest.java | 1 -
.../ignite/testframework/GridTestUtils.java | 2 +-
.../testframework/junits/GridAbstractTest.java | 2 -
.../junits/GridTestKernalContext.java | 26 +
.../IgniteCacheFullApiSelfTestSuite.java | 2 +-
.../testsuites/IgniteComputeGridTestSuite.java | 1 -
.../src/test/resources/load/html/index.html | 958 ----------
.../resources/load/js/gg-loadtest-server.js | 96 -
.../client/hadoop/GridHadoopClientProtocol.java | 4 +-
.../processors/hadoop/GridHadoopSetup.java | 2 +-
.../processors/hadoop/GridHadoopUtils.java | 8 +-
.../counter/GridHadoopFSCounterWriter.java | 2 +-
.../GridHadoopExternalTaskExecutor.java | 2 +-
.../GridGgfsHadoopDualAbstractSelfTest.java | 1 -
...idGgfsHadoopFileSystemHandshakeSelfTest.java | 1 -
...GgfsHadoopFileSystemLoggerStateSelfTest.java | 1 -
.../hadoop/GridHadoopAbstractSelfTest.java | 7 +-
.../hadoop/GridHadoopPopularWordsTest.java | 4 +-
.../processors/hadoop/GridHadoopStartup.java | 2 +-
.../h2/twostep/GridReduceQueryExecutor.java | 3 +
.../twostep/messages/GridNextPageRequest.java | 3 +
.../twostep/messages/GridNextPageResponse.java | 3 +
.../twostep/messages/GridQueryFailResponse.java | 3 +
.../h2/twostep/messages/GridQueryRequest.java | 3 +
.../log4j/GridLog4jCorrectFileNameTest.java | 1 -
.../http/jetty/GridJettyRestHandler.java | 6 +-
.../rest/protocols/http/jetty/rest.html | 4 +-
.../apache/ignite/schema/ui/SchemaLoadApp.java | 245 ++-
.../ignite/internal/GridFactorySelfTest.java | 11 +-
.../GridSpringBeanSerializationSelfTest.java | 2 -
.../config/VisorConfigurationCommand.scala | 4 -
.../scala/org/apache/ignite/visor/visor.scala | 8 +-
.../yardstick/config/benchmark-store.properties | 2 +-
modules/yardstick/config/ignite-base-config.xml | 2 -
.../yardstick/config/ignite-store-config.xml | 2 -
.../jdbc/IgniteJdbcStoreAbstractBenchmark.java | 2 +
pom.xml | 50 +-
170 files changed, 2642 insertions(+), 2883 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/571223f6/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcComplexQuerySelfTest.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/571223f6/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcLocalCachesSelfTest.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/571223f6/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcMetadataSelfTest.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/571223f6/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcPreparedStatementSelfTest.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/571223f6/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcResultSetSelfTest.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/571223f6/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcStatementSelfTest.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/571223f6/modules/clients/src/test/resources/spring-server-node.xml
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/571223f6/modules/clients/src/test/resources/spring-server-ssl-node.xml
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/571223f6/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/IgniteCache.java
index 3b26016,f100464..a438819
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
@@@ -244,15 -209,6 +245,13 @@@ public interface IgniteCache<K, V> exte
public Iterable<Entry<K, V>> localEntries(CachePeekMode... peekModes)
throws CacheException;
/**
+ * Gets query metrics.
+ *
+ * @return Metrics.
+ */
+ public QueryMetrics queryMetrics();
+
- public Map<K, V> localPartition(int part) throws CacheException;
-
+ /**
* Attempts to evict all entries associated with keys. Note,
* that entry will be evicted only if it's not used (not
* participating in any locks or transactions).
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/571223f6/modules/core/src/main/java/org/apache/ignite/cache/query/ContinuousQuery.java
----------------------------------------------------------------------
diff --cc
modules/core/src/main/java/org/apache/ignite/cache/query/ContinuousQuery.java
index efb448d,0000000..b02c65f
mode 100644,000000..100644
---
a/modules/core/src/main/java/org/apache/ignite/cache/query/ContinuousQuery.java
+++
b/modules/core/src/main/java/org/apache/ignite/cache/query/ContinuousQuery.java
@@@ -1,205 -1,0 +1,208 @@@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.*;
+
+import javax.cache.event.*;
+
+/**
+ * API for configuring and executing continuous cache queries.
+ * <p>
+ * Continuous queries are executed as follows:
+ * <ol>
+ * <li>
+ * Query is sent to requested grid nodes. Note that for {@link
org.apache.ignite.cache.CacheMode#LOCAL LOCAL}
+ * and {@link org.apache.ignite.cache.CacheMode#REPLICATED REPLICATED}
caches query will be always executed
+ * locally.
+ * </li>
+ * <li>
+ * Each node iterates through existing cache data and registers listeners
that will
+ * notify about further updates.
+ * <li>
+ * Each key-value pair is passed through optional filter and if this filter
returns
+ * true, key-value pair is sent to the master node (the one that executed
query).
+ * If filter is not provided, all pairs are sent.
+ * </li>
+ * <li>
+ * When master node receives key-value pairs, it notifies the local callback.
+ * </li>
+ * </ol>
+ * <h2 class="header">NOTE</h2>
+ * Under some concurrent circumstances callback may get several notifications
+ * for one cache update. This should be taken into account when implementing
callback.
+ * <h1 class="header">Query usage</h1>
+ * As an example, suppose we have cache with {@code 'Person'} objects and we
need
+ * to query all persons with salary above 1000.
+ * <p>
+ * Here is the {@code Person} class:
+ * <pre name="code" class="java">
+ * public class Person {
+ * // Name.
+ * private String name;
+ *
+ * // Salary.
+ * private double salary;
+ *
+ * ...
+ * }
+ * </pre>
+ * <p>
+ * You can create and execute continuous query like so:
+ * <pre name="code" class="java">
+ * // Create new continuous query.
+ * qry = cache.createContinuousQuery();
+ *
+ * // Callback that is called locally when update notifications are received.
+ * // It simply prints out information about all created persons.
+ * qry.callback(new GridPredicate2<UUID, Collection<Map.Entry<UUID,
Person>>>() {
+ * @Override public boolean apply(UUID uuid,
Collection<Map.Entry<UUID, Person>> entries) {
+ * for (Map.Entry<UUID, Person> e : entries) {
+ * Person p = e.getValue();
+ *
+ * X.println(">>>");
+ * X.println(">>> " + p.getFirstName() + " " +
p.getLastName() +
+ * "'s salary is " + p.getSalary());
+ * X.println(">>>");
+ * }
+ *
+ * return true;
+ * }
+ * });
+ *
+ * // This query will return persons with salary above 1000.
+ * qry.filter(new GridPredicate2<UUID, Person>() {
+ * @Override public boolean apply(UUID uuid, Person person) {
+ * return person.getSalary() > 1000;
+ * }
+ * });
+ *
+ * // Execute query.
+ * qry.execute();
+ * </pre>
+ * This will execute query on all nodes that have cache you are working with
and notify callback
+ * with both data that already exists in cache and further updates.
+ * <p>
+ * To stop receiving updates call {@link #close()} method:
+ * <pre name="code" class="java">
+ * qry.cancel();
+ * </pre>
+ * Note that one query instance can be executed only once. After it's
cancelled, it's non-operational.
+ * If you need to repeat execution, use {@link
org.apache.ignite.internal.processors.cache.query.CacheQueries#createContinuousQuery()}
method to create
+ * new query.
+ */
+public final class ContinuousQuery<K, V> extends Query<ContinuousQuery<K,V>>
implements AutoCloseable {
++ /** */
++ private static final long serialVersionUID = 0L;
++
+ /**
+ * Default buffer size. Size of {@code 1} means that all entries
+ * will be sent to master node immediately (buffering is disabled).
+ */
+ public static final int DFLT_BUF_SIZE = 1;
+
+ /** Maximum default time interval after which buffer will be flushed (if
buffering is enabled). */
+ public static final long DFLT_TIME_INTERVAL = 0;
+
+ /**
+ * Default value for automatic unsubscription flag. Remote filters
+ * will be unregistered by default if master node leaves topology.
+ */
+ public static final boolean DFLT_AUTO_UNSUBSCRIBE = true;
+
+ public void setInitialPredicate(Query filter) {
+ // TODO: implement.
+ }
+
+ /**
+ * Sets local callback. This callback is called only in local node when
new updates are received.
+ * <p> The callback predicate accepts ID of the node from where updates
are received and collection
+ * of received entries. Note that for removed entries value will be
{@code null}.
+ * <p>
+ * If the predicate returns {@code false}, query execution will be
cancelled.
+ * <p>
+ * <b>WARNING:</b> all operations that involve any kind of JVM-local or
distributed locking (e.g.,
+ * synchronization or transactional cache operations), should be executed
asynchronously without
+ * blocking the thread that called the callback. Otherwise, you can get
deadlocks.
+ *
+ * @param locLsnr Local callback.
+ */
+ public void setLocalListener(CacheEntryUpdatedListener<K, V> locLsnr) {
+ // TODO: implement.
+ }
+
+ /**
+ * Sets optional key-value filter. This filter is called before entry is
sent to the master node.
+ * <p>
+ * <b>WARNING:</b> all operations that involve any kind of JVM-local or
distributed locking
+ * (e.g., synchronization or transactional cache operations), should be
executed asynchronously
+ * without blocking the thread that called the filter. Otherwise, you can
get deadlocks.
+ *
+ * @param filter Key-value filter.
+ */
+ public void setRemoteFilter(CacheEntryEventFilter<K, V> filter) {
+ // TODO: implement.
+ }
+
+ /**
+ * Sets buffer size. <p> When a cache update happens, entry is first put
into a buffer. Entries from buffer will be
+ * sent to the master node only if the buffer is full or time provided
via {@link #timeInterval(long)} method is
+ * exceeded. <p> Default buffer size is {@code 1} which means that
entries will be sent immediately (buffering is
+ * disabled).
+ *
+ * @param bufSize Buffer size.
+ */
+ public void bufferSize(int bufSize) {
+ // TODO: implement.
+ }
+
+ /**
+ * Sets time interval. <p> When a cache update happens, entry is first
put into a buffer. Entries from buffer will
+ * be sent to the master node only if the buffer is full (its size can be
provided via {@link #bufferSize(int)}
+ * method) or time provided via this method is exceeded. <p> Default time
interval is {@code 0} which means that
+ * time check is disabled and entries will be sent only when buffer is
full.
+ *
+ * @param timeInterval Time interval.
+ */
+ public void timeInterval(long timeInterval) {
+ // TODO: implement.
+ }
+
+ /**
+ * Sets automatic unsubscribe flag. <p> This flag indicates that query
filters on remote nodes should be
+ * automatically unregistered if master node (node that initiated the
query) leaves topology. If this flag is {@code
+ * false}, filters will be unregistered only when the query is cancelled
from master node, and won't ever be
+ * unregistered if master node leaves grid. <p> Default value for this
flag is {@code true}.
+ *
+ * @param autoUnsubscribe Automatic unsubscription flag.
+ */
+ public void autoUnsubscribe(boolean autoUnsubscribe) {
+ // TODO: implement.
+ }
+
+ /**
+ * Stops continuous query execution. <p> Note that one query instance can
be executed only once. After it's
+ * cancelled, it's non-operational. If you need to repeat execution, use
{@link
+ *
org.apache.ignite.internal.processors.cache.query.CacheQueries#createContinuousQuery()}
method to create new query.
+ *
+ * @throws IgniteCheckedException In case of error.
+ */
+ @Override public void close() throws IgniteCheckedException {
+ // TODO: implement.
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/571223f6/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/571223f6/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index fa69d0c,7ef3c15..0601289
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@@ -1525,101 -1456,65 +1463,65 @@@ public class IgnitionEx
FailoverSpi[] failSpi = cfg.getFailoverSpi();
LoadBalancingSpi[] loadBalancingSpi = cfg.getLoadBalancingSpi();
SwapSpaceSpi swapspaceSpi = cfg.getSwapSpaceSpi();
- GridIndexingSpi indexingSpi = cfg.getIndexingSpi();
+ IndexingSpi indexingSpi = cfg.getIndexingSpi();
- execSvc = cfg.getExecutorService();
- sysExecSvc = cfg.getSystemExecutorService();
- p2pExecSvc = cfg.getPeerClassLoadingExecutorService();
- mgmtExecSvc = cfg.getManagementExecutorService();
- ggfsExecSvc = cfg.getGgfsExecutorService();
-
- if (execSvc == null) {
- isAutoExecSvc = true;
-
- execSvc = new IgniteThreadPoolExecutor(
- "pub-" + cfg.getGridName(),
- DFLT_PUBLIC_CORE_THREAD_CNT,
- DFLT_PUBLIC_MAX_THREAD_CNT,
- DFLT_PUBLIC_KEEP_ALIVE_TIME,
- new
LinkedBlockingQueue<Runnable>(DFLT_PUBLIC_THREADPOOL_QUEUE_CAP));
-
- // Pre-start all threads as they are guaranteed to be needed.
- ((ThreadPoolExecutor)execSvc).prestartAllCoreThreads();
- }
-
- if (sysExecSvc == null) {
- isAutoSysSvc = true;
-
- // Note that since we use 'LinkedBlockingQueue', number of
- // maximum threads has no effect.
- sysExecSvc = new IgniteThreadPoolExecutor(
- "sys-" + cfg.getGridName(),
- DFLT_SYSTEM_CORE_THREAD_CNT,
- DFLT_SYSTEM_MAX_THREAD_CNT,
- DFLT_SYSTEM_KEEP_ALIVE_TIME,
- new
LinkedBlockingQueue<Runnable>(DFLT_SYSTEM_THREADPOOL_QUEUE_CAP));
-
- // Pre-start all threads as they are guaranteed to be needed.
- ((ThreadPoolExecutor)sysExecSvc).prestartAllCoreThreads();
- }
-
- if (mgmtExecSvc == null) {
- isAutoMgmtSvc = true;
-
- // Note that since we use 'LinkedBlockingQueue', number of
- // maximum threads has no effect.
- // Note, that we do not pre-start threads here as management
pool may
- // not be needed.
- mgmtExecSvc = new IgniteThreadPoolExecutor(
- "mgmt-" + cfg.getGridName(),
- DFLT_MGMT_THREAD_CNT,
- DFLT_MGMT_THREAD_CNT,
- 0,
- new LinkedBlockingQueue<Runnable>());
- }
-
- if (p2pExecSvc == null) {
- isAutoP2PSvc = true;
-
- // Note that since we use 'LinkedBlockingQueue', number of
- // maximum threads has no effect.
- // Note, that we do not pre-start threads here as class
loading pool may
- // not be needed.
- p2pExecSvc = new IgniteThreadPoolExecutor(
- "p2p-" + cfg.getGridName(),
- DFLT_P2P_THREAD_CNT,
- DFLT_P2P_THREAD_CNT,
- 0,
- new LinkedBlockingQueue<Runnable>());
- }
-
- if (ggfsExecSvc == null) {
- isAutoGgfsSvc = true;
-
- int procCnt = Runtime.getRuntime().availableProcessors();
-
- // Note that we do not pre-start threads here as ggfs pool
may not be needed.
- ggfsExecSvc = new IgniteThreadPoolExecutor(
- "ggfs-" + cfg.getGridName(),
- procCnt,
- procCnt,
- 0,
- new LinkedBlockingQueue<Runnable>());
- }
-
- restExecSvc = clientCfg != null ?
clientCfg.getRestExecutorService() : null;
-
- if (restExecSvc != null && !cfg.isRestEnabled()) {
- U.warn(log, "REST executor service is configured, but REST is
disabled in configuration " +
- "(safely ignoring).");
- }
- else if (restExecSvc == null && clientCfg != null) {
- isAutoRestSvc = true;
+ execSvc = new IgniteThreadPoolExecutor(
+ "pub-" + cfg.getGridName(),
+ cfg.getPublicThreadPoolSize(),
+ cfg.getPublicThreadPoolSize(),
+ DFLT_PUBLIC_KEEP_ALIVE_TIME,
+ new
LinkedBlockingQueue<Runnable>(DFLT_PUBLIC_THREADPOOL_QUEUE_CAP));
+
+ // Pre-start all threads as they are guaranteed to be needed.
+ ((ThreadPoolExecutor) execSvc).prestartAllCoreThreads();
+
+ // Note that since we use 'LinkedBlockingQueue', number of
+ // maximum threads has no effect.
+ sysExecSvc = new IgniteThreadPoolExecutor(
+ "sys-" + cfg.getGridName(),
+ cfg.getSystemThreadPoolSize(),
+ cfg.getSystemThreadPoolSize(),
+ DFLT_SYSTEM_KEEP_ALIVE_TIME,
+ new
LinkedBlockingQueue<Runnable>(DFLT_SYSTEM_THREADPOOL_QUEUE_CAP));
+ // Pre-start all threads as they are guaranteed to be needed.
+ ((ThreadPoolExecutor) sysExecSvc).prestartAllCoreThreads();
+
+ // Note that since we use 'LinkedBlockingQueue', number of
+ // maximum threads has no effect.
+ // Note, that we do not pre-start threads here as management pool
may
+ // not be needed.
+ mgmtExecSvc = new IgniteThreadPoolExecutor(
+ "mgmt-" + cfg.getGridName(),
+ cfg.getManagementThreadPoolSize(),
+ cfg.getManagementThreadPoolSize(),
+ 0,
+ new LinkedBlockingQueue<Runnable>());
+
+ // Note that since we use 'LinkedBlockingQueue', number of
+ // maximum threads has no effect.
+ // Note, that we do not pre-start threads here as class loading
pool may
+ // not be needed.
+ p2pExecSvc = new IgniteThreadPoolExecutor(
+ "p2p-" + cfg.getGridName(),
+ cfg.getPeerClassLoadingThreadPoolSize(),
+ cfg.getPeerClassLoadingThreadPoolSize(),
+ 0,
+ new LinkedBlockingQueue<Runnable>());
+
+ // Note that we do not pre-start threads here as ggfs pool may
not be needed.
+ ggfsExecSvc = new IgniteThreadPoolExecutor(
+ "ggfs-" + cfg.getGridName(),
+ cfg.getGgfsThreadPoolSize(),
+ cfg.getGgfsThreadPoolSize(),
+ 0,
+ new LinkedBlockingQueue<Runnable>());
+
+ if (clientCfg != null) {
restExecSvc = new IgniteThreadPoolExecutor(
"rest-" + cfg.getGridName(),
- DFLT_REST_CORE_THREAD_CNT,
- DFLT_REST_MAX_THREAD_CNT,
+ clientCfg.getRestThreadPoolSize(),
+ clientCfg.getRestThreadPoolSize(),
DFLT_REST_KEEP_ALIVE_TIME,
new
LinkedBlockingQueue<Runnable>(DFLT_REST_THREADPOOL_QUEUE_CAP)
);
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/571223f6/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheWeakQueryIteratorsHolder.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/571223f6/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/571223f6/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java
----------------------------------------------------------------------
diff --cc
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java
index 0131fb0,e956b57..2ee690a
---
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java
+++
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java
@@@ -28,7 -28,10 +28,10 @@@ import java.io.*
*/
public class GridCacheSqlQuery implements Externalizable {
/** */
+ private static final long serialVersionUID = 0L;
+
+ /** */
- private static final Object[] EMPTY_PARAMS = {};
+ public static final Object[] EMPTY_PARAMS = {};
/** */
String alias;
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/571223f6/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/571223f6/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheLuceneQueryIndexTest.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/571223f6/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --cc
modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index 608f50f,911a2a2..acafddb
---
a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@@ -242,7 -229,10 +242,10 @@@ public class GridReduceQueryExecutor
/**
*
*/
- private static class Iter extends GridH2ResultSetIterator<List<?>>
implements GridCacheSqlResult {
+ private static class Iter extends GridH2ResultSetIterator<List<?>> {
+ /** */
+ private static final long serialVersionUID = 0L;
-
++
/**
* @param data Data array.
* @throws IgniteCheckedException If failed.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/571223f6/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridNextPageRequest.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/571223f6/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridNextPageResponse.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/571223f6/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryFailResponse.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/571223f6/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryRequest.java
----------------------------------------------------------------------