Repository: incubator-ignite
Updated Branches:
  refs/heads/sprint-1 f846f8a0c -> a8573b48a


ignite-sql-old -removed new api


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/6f2dc26c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/6f2dc26c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/6f2dc26c

Branch: refs/heads/sprint-1
Commit: 6f2dc26cb709c11ce433fbd4cfd2192923cf6d44
Parents: 7f062e6
Author: S.Vladykin <svlady...@gridgain.com>
Authored: Sat Feb 14 00:37:24 2015 +0300
Committer: S.Vladykin <svlady...@gridgain.com>
Committed: Sat Feb 14 00:37:24 2015 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/IgniteCache.java     |   9 -
 .../cache/query/QueryAffinityPredicate.java     | 135 ------------
 .../cache/query/QueryContinuousPredicate.java   | 215 -------------------
 .../apache/ignite/cache/query/QueryCursor.java  |  37 ----
 .../ignite/cache/query/QueryPredicate.java      |  76 -------
 .../apache/ignite/cache/query/QueryReducer.java |  30 ---
 .../ignite/cache/query/QuerySqlPredicate.java   | 118 ----------
 .../ignite/cache/query/QueryTextPredicate.java  |  90 --------
 .../query/annotations/QueryGroupIndex.java      |  49 -----
 .../cache/query/annotations/QuerySqlField.java  | 123 -----------
 .../query/annotations/QuerySqlFunction.java     |  67 ------
 .../cache/query/annotations/QueryTextField.java |  33 ---
 .../processors/cache/IgniteCacheProxy.java      |  33 ---
 13 files changed, 1015 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6f2dc26c/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java 
b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
index a2c5cc0..fabe3b5 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
@@ -18,7 +18,6 @@
 package org.apache.ignite;
 
 import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.query.*;
 import org.apache.ignite.cache.store.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.processors.cache.*;
@@ -191,17 +190,9 @@ public interface IgniteCache<K, V> extends 
javax.cache.Cache<K, V>, IgniteAsyncS
      */
     public boolean isLocalLocked(K key, boolean byCurrThread);
 
-    public QueryCursor<Entry<K, V>> query(QueryPredicate<K, V> filter);
 
-    public <R> QueryCursor<R> query(QueryReducer<Entry<K, V>, R> rmtRdc, 
QueryPredicate<K, V> filter);
 
-    public QueryCursor<List<?>> queryFields(QuerySqlPredicate<K, V> filter);
 
-    public <R> QueryCursor<R> queryFields(QueryReducer<List<?>, R> rmtRdc, 
QuerySqlPredicate<K, V> filter);
-
-    public QueryCursor<Entry<K, V>> localQuery(QueryPredicate<K, V> filter);
-
-    public QueryCursor<List<?>> localQueryFields(QuerySqlPredicate<K, V> 
filter);
 
     public Iterable<Entry<K, V>> localEntries(CachePeekMode... peekModes) 
throws CacheException;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6f2dc26c/modules/core/src/main/java/org/apache/ignite/cache/query/QueryAffinityPredicate.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryAffinityPredicate.java
 
b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryAffinityPredicate.java
deleted file mode 100644
index ea249b1..0000000
--- 
a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryAffinityPredicate.java
+++ /dev/null
@@ -1,135 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.cache.query;
-
-import org.apache.ignite.internal.util.typedef.internal.*;
-
-import javax.cache.*;
-
-/**
- * TODO: Add class description.
- *
- * @author @java.author
- * @version @java.version
- */
-public final class QueryAffinityPredicate<K, V> extends QueryPredicate<K, V> {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Predicate. */
-    private QueryPredicate<K, V> p;
-
-    /** Keys. */
-    private K[] keys;
-
-    /** Partitions. */
-    private int[] parts;
-
-    /**
-     * Empty constructor.
-     */
-    public QueryAffinityPredicate() {
-        // No-op.
-    }
-
-    /**
-     * Constructs affinity predicate with specified affinity keys.
-     *
-     * @param p Predicate.
-     * @param keys Affinity keys.
-     */
-    public QueryAffinityPredicate(QueryPredicate<K, V> p, K... keys) {
-        this.p = p;
-        this.keys = keys;
-    }
-
-    /**
-     * Constructs affinity predicate with specified affinity partitions.
-     *
-     * @param p Predicate.
-     * @param parts Affinity partitions.
-     */
-    public QueryAffinityPredicate(QueryPredicate<K, V> p, int[] parts) {
-        this.p = p;
-        this.parts = parts;
-    }
-
-    /**
-     * Gets wrapped predicate.
-     *
-     * @return Wrapped predicate.
-     */
-    public QueryPredicate<K, V> getPredicate() {
-        return p;
-    }
-
-    /**
-     * Sets wrapped predicate.
-     *
-     * @param p Wrapped predicate.
-     */
-    public void setPredicate(QueryPredicate<K, V> p) {
-        this.p = p;
-    }
-
-    /**
-     * Gets affinity keys.
-     *
-     * @return Affinity keys.
-     */
-    public K[] getKeys() {
-        return keys;
-    }
-
-    /**
-     * Sets affinity keys.
-     *
-     * @param keys Affinity keys.
-     */
-    public void setKeys(K... keys) {
-        this.keys = keys;
-    }
-
-    /**
-     * Gets affinity partitions.
-     *
-     * @return Affinity partitions.
-     */
-    public int[] getPartitions() {
-        return parts;
-    }
-
-    /**
-     * Sets affinity partitions.
-     *
-     * @param parts Affinity partitions.
-     */
-    public void setPartitions(int... parts) {
-        this.parts = parts;
-    }
-
-    /** {@inheritDoc} */
-    @Override public final boolean apply(Cache.Entry<K, V> entry) {
-        return p.apply(entry);
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(QueryAffinityPredicate.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6f2dc26c/modules/core/src/main/java/org/apache/ignite/cache/query/QueryContinuousPredicate.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryContinuousPredicate.java
 
b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryContinuousPredicate.java
deleted file mode 100644
index 01d5316..0000000
--- 
a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryContinuousPredicate.java
+++ /dev/null
@@ -1,215 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.cache.query;
-
-import org.apache.ignite.*;
-
-import javax.cache.*;
-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 
CacheQueries#createContinuousQuery()} method to create
- * new query.
- */
-// TODO: make class.
-public final class QueryContinuousPredicate<K, V> extends QueryPredicate<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(QueryPredicate<K, V> 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
-     * CacheQueries#createContinuousQuery()} method to create new query.
-     *
-     * @throws IgniteCheckedException In case of error.
-     */
-    @Override public void close() throws IgniteCheckedException {
-        // TODO: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean apply(Cache.Entry<K, V> entry) {
-        return false; // TODO: CODE: implement.
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6f2dc26c/modules/core/src/main/java/org/apache/ignite/cache/query/QueryCursor.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryCursor.java 
b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryCursor.java
deleted file mode 100644
index 9d7f64a..0000000
--- a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryCursor.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.cache.query;
-
-import java.util.*;
-
-/**
- * TODO: Add interface description.
- *
- * @author @java.author
- * @version @java.version
- */
-public interface QueryCursor<T> extends Iterable<T> {
-    /**
-     * Gets all query results and stores them in the collection.
-     * Use this method when you know in advance that query result is
-     * relatively small and will not cause memory utilization issues.
-     *
-     * @return Collection containing full query result.
-     */
-    public Collection<T> getAll();
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6f2dc26c/modules/core/src/main/java/org/apache/ignite/cache/query/QueryPredicate.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryPredicate.java 
b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryPredicate.java
deleted file mode 100644
index 31cf987..0000000
--- 
a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryPredicate.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.cache.query;
-
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.lang.*;
-
-import javax.cache.*;
-
-/**
- * Query predicate to pass into any of {@code Cache.query(...)} methods.
- * Use {@link QuerySqlPredicate} and {@link QueryTextPredicate} for SQL and
- * text queries accordingly.
- *
- * @author @java.author
- * @version @java.version
- */
-public abstract class QueryPredicate<K, V> implements 
IgnitePredicate<Cache.Entry<K, V>> {
-    /** Page size. */
-    private int pageSize;
-
-    /**
-     * Empty constructor.
-     */
-    protected QueryPredicate() {
-        // No-op.
-    }
-
-    /**
-     * Constructs query predicate with optional page size, if {@code 0},
-     * then {@link QueryConfiguration#getPageSize()} is used.
-     *
-     * @param pageSize Optional page size.
-     */
-    protected QueryPredicate(int pageSize) {
-        this.pageSize = pageSize;
-    }
-
-    /**
-     * Gets optional page size, if {@code 0}, then {@link 
QueryConfiguration#getPageSize()} is used.
-     *
-     * @return Optional page size.
-     */
-    public int getPageSize() {
-        return pageSize;
-    }
-
-    /**
-     * Sets optional page size, if {@code 0}, then {@link 
QueryConfiguration#getPageSize()} is used.
-     *
-     * @param pageSize Optional page size.
-     */
-    public void setPageSize(int pageSize) {
-        this.pageSize = pageSize;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(QueryPredicate.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6f2dc26c/modules/core/src/main/java/org/apache/ignite/cache/query/QueryReducer.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryReducer.java 
b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryReducer.java
deleted file mode 100644
index a01f4a9..0000000
--- a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryReducer.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.cache.query;
-
-/**
- * TODO: Add class description.
- *
- * @author @java.author
- * @version @java.version
- */
-public interface QueryReducer<T, R> {
-    public int collect(T entry);
-
-    public R reduce();
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6f2dc26c/modules/core/src/main/java/org/apache/ignite/cache/query/QuerySqlPredicate.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/query/QuerySqlPredicate.java
 
b/modules/core/src/main/java/org/apache/ignite/cache/query/QuerySqlPredicate.java
deleted file mode 100644
index 1f74786..0000000
--- 
a/modules/core/src/main/java/org/apache/ignite/cache/query/QuerySqlPredicate.java
+++ /dev/null
@@ -1,118 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.cache.query;
-
-import org.apache.ignite.internal.util.typedef.internal.*;
-
-import javax.cache.*;
-
-/**
- * Query SQL predicate to use with any of the {@code JCache.query(...)} and
- * {@code JCache.queryFields(...)} methods.
- *
- * @author @java.author
- * @version @java.version
- */
-public final class QuerySqlPredicate<K, V> extends QueryPredicate<K, V> {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** SQL clause. */
-    private String sql;
-
-    /** Arguments. */
-    private Object[] args;
-
-    /**
-     * Empty constructor.
-     */
-    public QuerySqlPredicate() {
-        // No-op.
-    }
-
-    /**
-     * Constructs SQL predicate with given SQL clause and arguments.
-     *
-     * @param sql SQL clause.
-     * @param args Arguments.
-     */
-    public QuerySqlPredicate(String sql, Object... args) {
-        this.sql = sql;
-        this.args = args;
-    }
-
-    /**
-     * Constructs SQL predicate with given SQL clause, page size, and 
arguments.
-     *
-     * @param sql SQL clause.
-     * @param pageSize Optional page size, if {@code 0}, then {@link 
QueryConfiguration#getPageSize()} is used.
-     * @param args Arguments.
-     */
-    public QuerySqlPredicate(String sql, int pageSize, Object[] args) {
-        super(pageSize);
-
-        this.sql = sql;
-        this.args = args;
-    }
-
-    /**
-     * Gets SQL clause.
-     *
-     * @return SQL clause.
-     */
-    public String getSql() {
-        return sql;
-    }
-
-    /**
-     * Sets SQL clause.
-     *
-     * @param sql SQL clause.
-     */
-    public void setSql(String sql) {
-        this.sql = sql;
-    }
-
-    /**
-     * Gets SQL arguments.
-     *
-     * @return SQL arguments.
-     */
-    public Object[] getArgs() {
-        return args;
-    }
-
-    /**
-     * Sets SQL arguments.
-     *
-     * @param args SQL arguments.
-     */
-    public void setArgs(Object... args) {
-        this.args = args;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean apply(Cache.Entry<K, V> entry) {
-        return false; // Not used.
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(QuerySqlPredicate.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6f2dc26c/modules/core/src/main/java/org/apache/ignite/cache/query/QueryTextPredicate.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryTextPredicate.java
 
b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryTextPredicate.java
deleted file mode 100644
index 1bdb10d..0000000
--- 
a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryTextPredicate.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.cache.query;
-
-import org.apache.ignite.internal.util.typedef.internal.*;
-
-import javax.cache.*;
-
-/**
- * TODO: Add class description.
- *
- * @author @java.author
- * @version @java.version
- */
-public final class QueryTextPredicate<K, V> extends QueryPredicate<K, V> {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** SQL clause. */
-    private String txt;
-
-    /** Arguments. */
-    private Object[] args;
-
-    public QueryTextPredicate(String txt, Object... args) {
-        this.txt = txt;
-        this.args = args;
-    }
-
-    /**
-     * Gets text search string.
-     *
-     * @return Text search string.
-     */
-    public String getText() {
-        return txt;
-    }
-
-    /**
-     * Sets text search string.
-     *
-     * @param txt Text search string.
-     */
-    public void setText(String txt) {
-        this.txt = txt;
-    }
-
-    /**
-     * Gets text search arguments.
-     *
-     * @return Text search arguments.
-     */
-    public Object[] getArgs() {
-        return args;
-    }
-
-    /**
-     * Sets text search arguments.
-     *
-     * @param args Text search arguments.
-     */
-    public void setArgs(Object... args) {
-        this.args = args;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean apply(Cache.Entry<K, V> entry) {
-        return false; // Not used.
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(QueryTextPredicate.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6f2dc26c/modules/core/src/main/java/org/apache/ignite/cache/query/annotations/QueryGroupIndex.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/query/annotations/QueryGroupIndex.java
 
b/modules/core/src/main/java/org/apache/ignite/cache/query/annotations/QueryGroupIndex.java
deleted file mode 100644
index 9c756c4..0000000
--- 
a/modules/core/src/main/java/org/apache/ignite/cache/query/annotations/QueryGroupIndex.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.cache.query.annotations;
-
-import java.lang.annotation.*;
-
-/**
- * Describes group index.
- */
-@Retention(RetentionPolicy.RUNTIME)
-@Target(ElementType.TYPE)
-public @interface QueryGroupIndex {
-    /**
-     * Group index name.
-     *
-     * @return Name.
-     */
-    String name();
-
-    /**
-     * List of group indexes for type.
-     */
-    @SuppressWarnings("PublicInnerClass")
-    @Retention(RetentionPolicy.RUNTIME)
-    @Target(ElementType.TYPE)
-    public static @interface List {
-        /**
-         * Gets array of group indexes.
-         *
-         * @return Array of group indexes.
-         */
-        QueryGroupIndex[] value();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6f2dc26c/modules/core/src/main/java/org/apache/ignite/cache/query/annotations/QuerySqlField.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/query/annotations/QuerySqlField.java
 
b/modules/core/src/main/java/org/apache/ignite/cache/query/annotations/QuerySqlField.java
deleted file mode 100644
index 6cb2593..0000000
--- 
a/modules/core/src/main/java/org/apache/ignite/cache/query/annotations/QuerySqlField.java
+++ /dev/null
@@ -1,123 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.cache.query.annotations;
-
-import java.lang.annotation.*;
-
-/**
- * Annotates fields for SQL queries. All fields that will be involved in SQL 
clauses must have
- * this annotation. For more information about cache queries see {@link 
org.apache.ignite.cache.query.CacheQuery} documentation.
- * @see org.apache.ignite.cache.query.CacheQuery
- */
-@Documented
-@Retention(RetentionPolicy.RUNTIME)
-@Target({ElementType.METHOD, ElementType.FIELD})
-public @interface QuerySqlField {
-    /**
-     * Specifies whether cache should maintain an index for this field or not.
-     * Just like with databases, field indexing may require additional overhead
-     * during updates, but makes select operations faster.
-     * <p>
-     * When indexed field is
-     * of type {@code com.vividsolutions.jts.geom.Geometry} (or any subclass 
of this class) then Ignite will
-     * consider this index as spatial providing performance boost for spatial 
queries.
-     *
-     * @return {@code True} if index must be created for this field in 
database.
-     */
-    boolean index() default false;
-
-    /**
-     * Specifies whether index should be in descending order or not. This 
property only
-     * makes sense if {@link #index()} property is set to {@code true}.
-     *
-     * @return {@code True} if field index should be in descending order.
-     */
-    boolean descending() default false;
-
-    /**
-     * Array of index groups this field belongs to. Groups are used for 
compound indexes,
-     * whenever index should be created on more than one field. All fields 
within the same
-     * group will belong to the same index.
-     * <p>
-     * Group indexes are needed because SQL engine can utilize only one index 
per table occurrence in a query.
-     * For example if we have two separate indexes on fields {@code a} and 
{@code b} of type {@code X} then
-     * query {@code select * from X where a = ? and b = ?} will use for 
filtering either index on field {@code a}
-     * or {@code b} but not both. For more effective query execution here it 
is preferable to have a single
-     * group index on both fields.
-     * <p>
-     * For more complex scenarios please refer to {@link QuerySqlField.Group} 
documentation.
-     *
-     * @return Array of group names.
-     */
-    String[] groups() default {};
-
-    /**
-     * Array of ordered index groups this field belongs to. For more 
information please refer to
-     * {@linkplain QuerySqlField.Group} documentation.
-     *
-     * @return Array of ordered group indexes.
-     * @see #groups()
-     */
-    Group[] orderedGroups() default {};
-
-    /**
-     * Property name. If not provided then field name will be used.
-     *
-     * @return Name of property.
-     */
-    String name() default "";
-
-    /**
-     * Describes group of index and position of field in this group.
-     * <p>
-     * Opposite to {@link #groups()} this annotation gives control over order 
of fields in a group index.
-     * This can be needed in scenarios when we have a query like
-     * {@code select * from X where a = ? and b = ? order by b desc}. If we 
have index {@code (a asc, b asc)}
-     * sorting on {@code b} will be performed. Here it is preferable to have 
index {@code (b desc, a asc)}
-     * which will still allow query to search on index using both fields and 
avoid sorting because index
-     * is already sorted in needed way.
-     *
-     * @see #groups()
-     * @see #orderedGroups()
-     */
-    @Retention(RetentionPolicy.RUNTIME)
-    @Target({ElementType.METHOD, ElementType.FIELD})
-    @SuppressWarnings("PublicInnerClass")
-    public static @interface Group {
-        /**
-         * Group index name where this field participate.
-         *
-         * @return Group index name
-         */
-        String name();
-
-        /**
-         * Fields in this group index will be sorted on this attribute.
-         *
-         * @return Order number.
-         */
-        int order();
-
-        /**
-         * Defines sorting order for this field in group.
-         *
-         * @return True if field will be in descending order.
-         */
-        boolean descending() default false;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6f2dc26c/modules/core/src/main/java/org/apache/ignite/cache/query/annotations/QuerySqlFunction.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/query/annotations/QuerySqlFunction.java
 
b/modules/core/src/main/java/org/apache/ignite/cache/query/annotations/QuerySqlFunction.java
deleted file mode 100644
index 7573162..0000000
--- 
a/modules/core/src/main/java/org/apache/ignite/cache/query/annotations/QuerySqlFunction.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.cache.query.annotations;
-
-import java.lang.annotation.*;
-
-/**
- * Annotates public static methods in classes to be used in SQL queries as 
custom functions.
- * Annotated class must be registered in H2 indexing SPI using following method
- * {@link 
org.apache.ignite.configuration.QueryConfiguration#setIndexCustomFunctionClasses(java.lang.Class[])}.
- * <p>
- * Example usage:
- * <pre name="code" class="java">
- *     public class MyFunctions {
- *         &#64;CacheQuerySqlFunction
- *         public static int sqr(int x) {
- *             return x * x;
- *         }
- *     }
- *
- *     // Register.
- *     indexing.setIndexCustomFunctionClasses(MyFunctions.class);
- *
- *     // And use in queries.
- *     cache.queries().createSqlFieldsQuery("select sqr(2) where sqr(1) = 1");
- * </pre>
- * <p>
- * For more information about H2 custom functions please refer to
- * <a 
href="http://www.h2database.com/html/features.html#user_defined_functions";>H2 
documentation</a>.
- */
-@Documented
-@Retention(RetentionPolicy.RUNTIME)
-@Target(ElementType.METHOD)
-public @interface QuerySqlFunction {
-    /**
-     * Specifies alias for the function to be used form SQL queries.
-     * If no alias provided method name will be used.
-     *
-     * @return Alias for function.
-     */
-    String alias() default "";
-
-    /**
-     * Specifies if the function is deterministic (result depends only on 
input parameters).
-     * <p>
-     * Deterministic function is a function which always returns the same 
result
-     * assuming that input parameters are the same.
-     *
-     * @return {@code true} If function is deterministic, {@code false} 
otherwise.
-     */
-    boolean deterministic() default false;
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6f2dc26c/modules/core/src/main/java/org/apache/ignite/cache/query/annotations/QueryTextField.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/query/annotations/QueryTextField.java
 
b/modules/core/src/main/java/org/apache/ignite/cache/query/annotations/QueryTextField.java
deleted file mode 100644
index 3015073..0000000
--- 
a/modules/core/src/main/java/org/apache/ignite/cache/query/annotations/QueryTextField.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.cache.query.annotations;
-
-import java.lang.annotation.*;
-
-/**
- * Annotation for fields or getters to be indexed for full text
- * search using {@code H2 TEXT} indexing. For more information
- * refer to {@link org.apache.ignite.cache.query.CacheQuery} documentation.
- * @see org.apache.ignite.cache.query.CacheQuery
- */
-@Documented
-@Retention(RetentionPolicy.RUNTIME)
-@Target({ElementType.METHOD, ElementType.FIELD, ElementType.TYPE})
-public @interface QueryTextField {
-    // No-op.
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6f2dc26c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
index db9fbe0..9238d3c 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.query.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.future.*;
@@ -934,41 +933,9 @@ public class IgniteCacheProxy<K, V> extends 
AsyncSupportAdapter<IgniteCache<K, V
         }
     }
 
-    /** {@inheritDoc} */
-    @Override public QueryCursor<Entry<K, V>> query(QueryPredicate<K, V> 
filter) {
-        // TODO IGNITE-1.
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
-    @Override public <R> QueryCursor<R> query(QueryReducer<Entry<K, V>, R> 
rmtRdc, QueryPredicate<K, V> filter) {
-        // TODO IGNITE-1.
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
-    @Override public QueryCursor<List<?>> queryFields(QuerySqlPredicate<K, V> 
filter) {
-        // TODO IGNITE-1.
-        throw new UnsupportedOperationException();
-    }
 
-    /** {@inheritDoc} */
-    @Override public <R> QueryCursor<R> queryFields(QueryReducer<List<?>, R> 
rmtRdc, QuerySqlPredicate<K, V> filter) {
-        // TODO IGNITE-1.
-        throw new UnsupportedOperationException();
-    }
 
-    /** {@inheritDoc} */
-    @Override public QueryCursor<Entry<K, V>> localQuery(QueryPredicate<K, V> 
filter) {
-        // TODO IGNITE-1.
-        throw new UnsupportedOperationException();
-    }
 
-    /** {@inheritDoc} */
-    @Override public QueryCursor<List<?>> 
localQueryFields(QuerySqlPredicate<K, V> filter) {
-        // TODO IGNITE-1.
-        throw new UnsupportedOperationException();
-    }
 
     /** {@inheritDoc} */
     @Override protected IgniteCache<K, V> createAsyncInstance() {

Reply via email to