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&lt;UUID, Collection&lt;Map.Entry&lt;UUID, 
Person&gt;&gt;&gt;() {
 + *     &#64;Override public boolean apply(UUID uuid, 
Collection&lt;Map.Entry&lt;UUID, Person&gt;&gt; entries) {
 + *         for (Map.Entry&lt;UUID, Person&gt; e : entries) {
 + *             Person p = e.getValue();
 + *
 + *             X.println("&gt;&gt;&gt;");
 + *             X.println("&gt;&gt;&gt; " + p.getFirstName() + " " + 
p.getLastName() +
 + *                 "'s salary is " + p.getSalary());
 + *             X.println("&gt;&gt;&gt;");
 + *         }
 + *
 + *         return true;
 + *     }
 + * });
 + *
 + * // This query will return persons with salary above 1000.
 + * qry.filter(new GridPredicate2&lt;UUID, Person&gt;() {
 + *     &#64;Override public boolean apply(UUID uuid, Person person) {
 + *         return person.getSalary() &gt; 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
----------------------------------------------------------------------

Reply via email to