You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2016/08/15 12:12:36 UTC

ignite git commit: ignite-3407 HTTP REST: query commands without pageSize failed with NPE

Repository: ignite
Updated Branches:
  refs/heads/master a235985ef -> bfa375bbc


ignite-3407 HTTP REST: query commands without pageSize failed with NPE


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

Branch: refs/heads/master
Commit: bfa375bbc991c1d0eea9837952be5fee87e4b558
Parents: a235985ef
Author: samaitra <sa...@gmail.com>
Authored: Mon Aug 15 14:59:42 2016 +0300
Committer: agura <ag...@gridgain.com>
Committed: Mon Aug 15 15:12:17 2016 +0300

----------------------------------------------------------------------
 .../handlers/query/QueryCommandHandler.java     |  16 +-
 .../rest/request/RestQueryRequest.java          |   2 +-
 .../query/GridQueryCommandHandlerTest.java      | 191 +++++++++++++++++++
 .../testsuites/IgniteRestHandlerTestSuite.java  |   2 +
 4 files changed, 207 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/bfa375bb/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/query/QueryCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/query/QueryCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/query/QueryCommandHandler.java
index 4317dd9..ee728a6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/query/QueryCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/query/QueryCommandHandler.java
@@ -30,6 +30,7 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.ReentrantLock;
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.cache.query.Query;
 import org.apache.ignite.cache.query.QueryCursor;
@@ -217,22 +218,31 @@ public class QueryCommandHandler extends GridRestCommandHandlerAdapter {
         assert SUPPORTED_COMMANDS.contains(req.command());
         assert req instanceof RestQueryRequest : "Invalid type of query request.";
 
+        if (req.command() != CLOSE_SQL_QUERY) {
+            Integer pageSize = ((RestQueryRequest) req).pageSize();
+
+            if (pageSize == null)
+                return new GridFinishedFuture<>(
+                        new IgniteCheckedException(GridRestCommandHandlerAdapter.missingParameter("pageSize"))
+                );
+        }
+
         switch (req.command()) {
             case EXECUTE_SQL_QUERY:
             case EXECUTE_SQL_FIELDS_QUERY:
             case EXECUTE_SCAN_QUERY: {
                 return ctx.closure().callLocalSafe(
-                    new ExecuteQueryCallable(ctx, (RestQueryRequest)req, qryCurs), false);
+                        new ExecuteQueryCallable(ctx, (RestQueryRequest) req, qryCurs), false);
             }
 
             case FETCH_SQL_QUERY: {
                 return ctx.closure().callLocalSafe(
-                    new FetchQueryCallable((RestQueryRequest)req, qryCurs), false);
+                        new FetchQueryCallable((RestQueryRequest) req, qryCurs), false);
             }
 
             case CLOSE_SQL_QUERY: {
                 return ctx.closure().callLocalSafe(
-                    new CloseQueryCallable((RestQueryRequest)req, qryCurs), false);
+                        new CloseQueryCallable((RestQueryRequest) req, qryCurs), false);
             }
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bfa375bb/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestQueryRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestQueryRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestQueryRequest.java
index 7159c83..75c74db 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestQueryRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestQueryRequest.java
@@ -88,7 +88,7 @@ public class RestQueryRequest extends GridRestRequest {
     /**
      * @return Page size.
      */
-    public int pageSize() {
+    public Integer pageSize() {
         return pageSize;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bfa375bb/modules/core/src/test/java/org/apache/ignite/internal/processors/rest/handlers/query/GridQueryCommandHandlerTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/rest/handlers/query/GridQueryCommandHandlerTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/rest/handlers/query/GridQueryCommandHandlerTest.java
new file mode 100644
index 0000000..7e4cd82
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/rest/handlers/query/GridQueryCommandHandlerTest.java
@@ -0,0 +1,191 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.rest.handlers.query;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.configuration.ConnectorConfiguration;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.rest.GridRestCommand;
+import org.apache.ignite.internal.processors.rest.GridRestResponse;
+import org.apache.ignite.internal.processors.rest.request.RestQueryRequest;
+import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
+import org.apache.ignite.testframework.junits.GridTestKernalContext;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import java.util.Collection;
+
+/**
+ * REST query command handler tests.
+ */
+public class GridQueryCommandHandlerTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGrid();
+
+        ConnectorConfiguration connCfg = new ConnectorConfiguration();
+
+        connCfg.setIdleQueryCursorCheckFrequency(1000);
+        connCfg.setIdleQueryCursorTimeout(1000);
+
+        grid().configuration().setConnectorConfiguration(connCfg);
+
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSupportedCommands() throws Exception {
+        GridTestKernalContext ctx = newContext(grid().configuration());
+
+        ctx.add(new GridTimeoutProcessor(ctx));
+
+        QueryCommandHandler cmdHnd = new QueryCommandHandler(ctx);
+
+        Collection<GridRestCommand> commands = cmdHnd.supportedCommands();
+
+        assertEquals(5, commands.size());
+
+        assertTrue(commands.contains(GridRestCommand.EXECUTE_SQL_QUERY));
+        assertTrue(commands.contains(GridRestCommand.EXECUTE_SQL_FIELDS_QUERY));
+        assertTrue(commands.contains(GridRestCommand.EXECUTE_SCAN_QUERY));
+        assertTrue(commands.contains(GridRestCommand.FETCH_SQL_QUERY));
+        assertTrue(commands.contains(GridRestCommand.CLOSE_SQL_QUERY));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testUnsupportedCommands() throws Exception {
+        GridTestKernalContext ctx = newContext(grid().configuration());
+
+        ctx.add(new GridTimeoutProcessor(ctx));
+
+        QueryCommandHandler cmdHnd = new QueryCommandHandler(ctx);
+
+        Collection<GridRestCommand> commands = cmdHnd.supportedCommands();
+
+        assertFalse(commands.contains(GridRestCommand.LOG));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNullCache() throws Exception {
+        QueryCommandHandler cmdHnd = new QueryCommandHandler(grid().context());
+
+        Integer arg1 = 1000;
+
+        Object[] arr = new Object[] {arg1, arg1};
+
+        RestQueryRequest req = new RestQueryRequest();
+
+        req.command(GridRestCommand.EXECUTE_SQL_QUERY);
+        req.queryType(RestQueryRequest.QueryType.SCAN);
+        req.typeName(Integer.class.getName());
+        req.pageSize(10);
+        req.sqlQuery("salary+>+%3F+and+salary+<%3D+%3F");
+        req.arguments(arr);
+        req.cacheName(null);
+
+        IgniteInternalFuture<GridRestResponse> resp = cmdHnd.handleAsync(req);
+        resp.get();
+
+        assertEquals("Failed to find cache with name: null", resp.result().getError());
+        assertEquals(GridRestResponse.STATUS_FAILED, resp.result().getSuccessStatus());
+        assertNull(resp.result().getResponse());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNullPageSize() throws Exception {
+        grid().getOrCreateCache(getName());
+
+        QueryCommandHandler cmdHnd = new QueryCommandHandler(grid().context());
+
+        Integer arg1 = 1000;
+
+        Object[] arr = new Object[] {arg1, arg1};
+
+        RestQueryRequest req = new RestQueryRequest();
+
+        req.command(GridRestCommand.EXECUTE_SQL_QUERY);
+        req.queryType(RestQueryRequest.QueryType.SCAN);
+        req.typeName(Integer.class.getName());
+
+        req.pageSize(null);
+        req.sqlQuery("salary+>+%3F+and+salary+<%3D+%3F");
+
+        req.arguments(arr);
+        req.cacheName(getName());
+
+        try {
+            IgniteInternalFuture<GridRestResponse> resp = cmdHnd.handleAsync(req);
+            resp.get();
+
+            fail("Expected exception not thrown.");
+        }
+        catch (IgniteCheckedException e) {
+            info("Got expected exception: " + e);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testQuery() throws Exception {
+        grid().getOrCreateCache(getName());
+
+        QueryCommandHandler cmdHnd = new QueryCommandHandler(grid().context());
+
+        Integer arg1 = 1000;
+
+        Object[] arr = new Object[] {arg1, arg1};
+
+        RestQueryRequest req = new RestQueryRequest();
+
+        req.command(GridRestCommand.EXECUTE_SQL_QUERY);
+        req.queryType(RestQueryRequest.QueryType.SCAN);
+        req.typeName(Integer.class.getName());
+        req.pageSize(null);
+        req.sqlQuery("salary+>+%3F+and+salary+<%3D+%3F");
+        req.arguments(arr);
+        req.cacheName(getName());
+        req.pageSize(10);
+
+        IgniteInternalFuture<GridRestResponse> resp = cmdHnd.handleAsync(req);
+        resp.get();
+
+        assertNull(resp.result().getError());
+        assertEquals(GridRestResponse.STATUS_SUCCESS, resp.result().getSuccessStatus());
+        assertNotNull(resp.result().getResponse());
+
+        CacheQueryResult res = (CacheQueryResult) resp.result().getResponse();
+
+        assertTrue(res.getLast());
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/bfa375bb/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteRestHandlerTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteRestHandlerTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteRestHandlerTestSuite.java
index 42c6752..6263e8b 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteRestHandlerTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteRestHandlerTestSuite.java
@@ -21,6 +21,7 @@ import junit.framework.TestSuite;
 import org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheAtomicCommandHandlerSelfTest;
 import org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandlerSelfTest;
 import org.apache.ignite.internal.processors.rest.handlers.log.GridLogCommandHandlerTest;
+import org.apache.ignite.internal.processors.rest.handlers.query.GridQueryCommandHandlerTest;
 
 /**
  * REST support tests.
@@ -36,6 +37,7 @@ public class IgniteRestHandlerTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheCommandHandlerSelfTest.class);
         suite.addTestSuite(GridCacheAtomicCommandHandlerSelfTest.class);
         suite.addTestSuite(GridLogCommandHandlerTest.class);
+        suite.addTestSuite(GridQueryCommandHandlerTest.class);
 
         return suite;
     }