[
https://issues.apache.org/jira/browse/NIFI-5642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16664277#comment-16664277
]
ASF GitHub Bot commented on NIFI-5642:
--------------------------------------
Github user mattyb149 commented on a diff in the pull request:
https://github.com/apache/nifi/pull/3051#discussion_r228332165
--- Diff:
nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/QueryCassandra.java
---
@@ -400,77 +478,87 @@ public static long convertToJsonStream(final
ResultSet rs, final OutputStream ou
outStream.write("{\"results\":[".getBytes(charset));
final ColumnDefinitions columnDefinitions =
rs.getColumnDefinitions();
long nrOfRows = 0;
+ long rowsAvailableWithoutFetching =
rs.getAvailableWithoutFetching();
+
if (columnDefinitions != null) {
- do {
-
- // Grab the ones we have
- int rowsAvailableWithoutFetching =
rs.getAvailableWithoutFetching();
- if (rowsAvailableWithoutFetching == 0) {
- // Get more
- if (timeout <= 0 || timeUnit == null) {
- rs.fetchMoreResults().get();
- } else {
- rs.fetchMoreResults().get(timeout, timeUnit);
- }
+
+ // Grab the ones we have
+ if (rowsAvailableWithoutFetching == 0) {
+ // Get more
+ if (timeout <= 0 || timeUnit == null) {
+ rs.fetchMoreResults().get();
+ } else {
+ rs.fetchMoreResults().get(timeout, timeUnit);
}
+ rowsAvailableWithoutFetching =
rs.getAvailableWithoutFetching();
+ }
- for (Row row : rs) {
- if (nrOfRows != 0) {
+ if(maxRowsPerFlowFile == 0){
+ maxRowsPerFlowFile = rowsAvailableWithoutFetching;
+ }
+ Row row;
+ while(nrOfRows < maxRowsPerFlowFile){
+ try {
+ row = rs.iterator().next();
+ }catch (NoSuchElementException nsee){
+ //nrOfRows -= 1;
--- End diff --
This is commented out here but active in the Avro version above, I assume
they need to be the same?
> QueryCassandra processor : output FlowFiles as soon fetch_size is reached
> -------------------------------------------------------------------------
>
> Key: NIFI-5642
> URL: https://issues.apache.org/jira/browse/NIFI-5642
> Project: Apache NiFi
> Issue Type: Bug
> Affects Versions: 1.7.1
> Reporter: André Gomes Lamas Otero
> Priority: Major
>
> When I'm using QueryCassandra alongside with fetch_size parameter I expected
> that as soon my reader reaches the fetch_size the processor outputs some data
> to be processed by the next processor, but QueryCassandra reads all the data,
> then output the flow files.
> I'll start to work on a patch for this situation, I'll appreciate any
> suggestion.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)