From 88682359937afbc0e48df769c296606316c7b1d0 Mon Sep 17 00:00:00 2001 From: Byunghwa Yun Date: Thu, 22 Sep 2016 19:00:16 +0900 Subject: [PATCH] NIFI-2803 The pagination of GenerateTableFetch has a sort bug. Signed-off-by: Matt Burgess NIFI-2803: Updated unit tests for GenerateTableFetch Signed-off-by: Matt Burgess This closes #1052 --- .../processors/standard/GenerateTableFetch.java | 2 +- .../standard/TestGenerateTableFetch.java | 16 ++++++++-------- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GenerateTableFetch.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GenerateTableFetch.java index f1c86f8525..bff1024ae1 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GenerateTableFetch.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GenerateTableFetch.java @@ -230,7 +230,7 @@ public class GenerateTableFetch extends AbstractDatabaseFetchProcessor { Integer limit = partitionSize == 0 ? null : partitionSize; Integer offset = partitionSize == 0 ? null : i * partitionSize; - final String query = dbAdapter.getSelectStatement(tableName, columnNames, StringUtils.join(maxValueClauses, " AND "), null, limit, offset); + final String query = dbAdapter.getSelectStatement(tableName, columnNames, whereClause, StringUtils.join(maxValueColumnNameList, ", "), limit, offset); sqlFlowFile = session.create(); sqlFlowFile = session.write(sqlFlowFile, out -> { out.write(query.getBytes()); diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestGenerateTableFetch.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestGenerateTableFetch.java index 3b9f3a354e..8a8aa01220 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestGenerateTableFetch.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestGenerateTableFetch.java @@ -128,7 +128,7 @@ public class TestGenerateTableFetch { runner.assertAllFlowFilesTransferred(REL_SUCCESS, 1); MockFlowFile flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(0); String query = new String(flowFile.toByteArray()); - assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE FETCH NEXT 10000 ROWS ONLY", query); + assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE ORDER BY ID FETCH NEXT 10000 ROWS ONLY", query); ResultSet resultSet = stmt.executeQuery(query); // Should be three records assertTrue(resultSet.next()); @@ -153,7 +153,7 @@ public class TestGenerateTableFetch { // Verify first flow file's contents flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(0); query = new String(flowFile.toByteArray()); - assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE WHERE ID > 2 FETCH NEXT 2 ROWS ONLY", query); + assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE WHERE ID > 2 ORDER BY ID FETCH NEXT 2 ROWS ONLY", query); resultSet = stmt.executeQuery(query); // Should be two records assertTrue(resultSet.next()); @@ -163,7 +163,7 @@ public class TestGenerateTableFetch { // Verify second flow file's contents flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(1); query = new String(flowFile.toByteArray()); - assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE WHERE ID > 2 OFFSET 2 ROWS FETCH NEXT 2 ROWS ONLY", query); + assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE WHERE ID > 2 ORDER BY ID OFFSET 2 ROWS FETCH NEXT 2 ROWS ONLY", query); resultSet = stmt.executeQuery(query); // Should be one record assertTrue(resultSet.next()); @@ -176,7 +176,7 @@ public class TestGenerateTableFetch { runner.assertAllFlowFilesTransferred(REL_SUCCESS, 1); flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(0); query = new String(flowFile.toByteArray()); - assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE WHERE ID > 5 FETCH NEXT 2 ROWS ONLY", query); + assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE WHERE ID > 5 ORDER BY ID FETCH NEXT 2 ROWS ONLY", query); resultSet = stmt.executeQuery(query); // Should be one record assertTrue(resultSet.next()); @@ -189,13 +189,13 @@ public class TestGenerateTableFetch { runner.run(); runner.assertAllFlowFilesTransferred(REL_SUCCESS, 4); // 7 records with partition size 2 means 4 generated FlowFiles flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(0); - assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE FETCH NEXT 2 ROWS ONLY", new String(flowFile.toByteArray())); + assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE ORDER BY name FETCH NEXT 2 ROWS ONLY", new String(flowFile.toByteArray())); flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(1); - assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE OFFSET 2 ROWS FETCH NEXT 2 ROWS ONLY", new String(flowFile.toByteArray())); + assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE ORDER BY name OFFSET 2 ROWS FETCH NEXT 2 ROWS ONLY", new String(flowFile.toByteArray())); flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(2); - assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE OFFSET 4 ROWS FETCH NEXT 2 ROWS ONLY", new String(flowFile.toByteArray())); + assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE ORDER BY name OFFSET 4 ROWS FETCH NEXT 2 ROWS ONLY", new String(flowFile.toByteArray())); flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(3); - assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE OFFSET 6 ROWS FETCH NEXT 2 ROWS ONLY", new String(flowFile.toByteArray())); + assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE ORDER BY name OFFSET 6 ROWS FETCH NEXT 2 ROWS ONLY", new String(flowFile.toByteArray())); runner.clearTransferState(); }