You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by markap14 <gi...@git.apache.org> on 2017/04/25 21:20:06 UTC

[GitHub] nifi pull request #1695: NIFI-3739: Added ConsumeKafkaRecord_0_10 and Publis...

GitHub user markap14 opened a pull request:

    https://github.com/apache/nifi/pull/1695

    NIFI-3739: Added ConsumeKafkaRecord_0_10 and PublishKafkaRecord_0_10 \u2026

    \u2026processors
    
    Thank you for submitting a contribution to Apache NiFi.
    
    In order to streamline the review of the contribution we ask you
    to ensure the following steps have been taken:
    
    ### For all changes:
    - [ ] Is there a JIRA ticket associated with this PR? Is it referenced 
         in the commit message?
    
    - [ ] Does your PR title start with NIFI-XXXX where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
    
    - [ ] Has your PR been rebased against the latest commit within the target branch (typically master)?
    
    - [ ] Is your initial contribution a single, squashed commit?
    
    ### For code changes:
    - [ ] Have you ensured that the full suite of tests is executed via mvn -Pcontrib-check clean install at the root nifi folder?
    - [ ] Have you written or updated unit tests to verify your changes?
    - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)? 
    - [ ] If applicable, have you updated the LICENSE file, including the main LICENSE file under nifi-assembly?
    - [ ] If applicable, have you updated the NOTICE file, including the main NOTICE file found under nifi-assembly?
    - [ ] If adding new Properties, have you added .displayName in addition to .name (programmatic access) for each of the new properties?
    
    ### For documentation related changes:
    - [ ] Have you ensured that format looks appropriate for the output in which it is rendered?
    
    ### Note:
    Please ensure that once the PR is submitted, you check travis-ci for build issues and submit an update to your PR as soon as possible.


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/markap14/nifi NIFI-3739

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/nifi/pull/1695.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #1695
    
----
commit 6e12aeb18b8cdcc9becfac78cee588c27edb4bc8
Author: Mark Payne <ma...@hotmail.com>
Date:   2017-04-25T21:19:41Z

    NIFI-3739: Added ConsumeKafkaRecord_0_10 and PublishKafkaRecord_0_10 processors

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1695: NIFI-3739: Added ConsumeKafkaRecord_0_10 and Publis...

Posted by joewitt <gi...@git.apache.org>.
Github user joewitt commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1695#discussion_r114150476
  
    --- Diff: nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java ---
    @@ -392,13 +394,27 @@ private void writeDemarcatedData(final ProcessSession session, final List<Consum
             bundleMap.put(topicPartition, tracker);
         }
     
    +    private void rollback(final TopicPartition topicPartition) {
    +        final OffsetAndMetadata offsetAndMetadata = kafkaConsumer.committed(topicPartition);
    +        final long offset = offsetAndMetadata.offset();
    +        kafkaConsumer.seek(topicPartition, offset);
    +    }
    +
         private void writeRecordData(final ProcessSession session, final List<ConsumerRecord<byte[], byte[]>> records, final TopicPartition topicPartition) {
             FlowFile flowFile = session.create();
             try {
                 final RecordSetWriter writer;
                 try {
                     writer = writerFactory.createWriter(logger, flowFile, new ByteArrayInputStream(new byte[0]));
                 } catch (final Exception e) {
    +                logger.error(
    +                    "Failed to obtain a Record Writer for serializing Kafka messages. This generally happens because the "
    +                        + "Record Writer cannot obtain the appropriate Schema, due to failure to connect to a remote Schema Registry "
    +                        + "or due to the Schema Access Strategy being dependent upon FlowFile Attributes that are not available. "
    +                        + "Will roll back the Kafka session.", e);
    +
    +                rollback(topicPartition);
    --- End diff --
    
    actually since we've not yet committed our offsets this is probably client side only...so probably fine


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1695: NIFI-3739: Added ConsumeKafkaRecord_0_10 and PublishKafkaR...

Posted by joewitt <gi...@git.apache.org>.
Github user joewitt commented on the issue:

    https://github.com/apache/nifi/pull/1695
  
    one more observation and cannot recall if I mentioned this yet.  But here is some original JSON we'd get from provenance.  This is before turning into avro and then turning back into json.
    
    {code}
    {
    "eventId":"3ddbed75-c436-4112-a2bd-bbdd05320b23",
    "eventOrdinal":13720400,
    "eventType":"CREATE",
    "timestampMillis":1493439571667,
    "timestamp":"2017-04-29T04:19:31.667Z",
    "durationMillis":-1,
    "lineageStart":1493439571667,
    "componentId":"b259f36c-015b-1000-3ca6-c664e156cdf7",
    "componentType":"GenerateFlowFile",
    "componentName":"GenerateFlowFile",
    "entityId":"2ab3c2d8-cb45-44ae-bd45-72080aff3e78",
    "entityType":"org.apache.nifi.flowfile.FlowFile",
    "entitySize":0,
    "updatedAttributes":
    {
        "path":"./",
        "uuid":"2ab3c2d8-cb45-44ae-bd45-72080aff3e78",
        "filename":"464576540850588"
    },
    "previousAttributes":{},
    "actorHostname":"10.0.0.15",
    "contentURI":"http://10.0.0.15:8080/nifi-api/provenance-events/13720400/content/output",
    "previousContentURI":"http://10.0.0.15:8080/nifi-api/provenance-events/13720400/content/input",
    "parentIds":[],
    "childIds":[],
    "platform":"nifi",
    "application":"NiFi Flow"
    }
    {code}
    
    Here is the after avro and then back to json view
    
    {code}
    {
      "eventId" : "edf16cc9-b50d-4cbb-9356-58eb4cf8fd3e",
      "eventOrdinal" : 13585491,
      "eventType" : "CREATE",
      "timestampMillis" : "2017-04-29 04:18:50",
      "durationMillis" : -1,
      "lineageStart" : "2017-04-29 04:18:50",
      "details" : null,
      "componentId" : "b259f36c-015b-1000-3ca6-c664e156cdf7",
      "componentType" : "GenerateFlowFile",
      "componentName" : "GenerateFlowFile",
      "entityId" : "ef12f52c-4ae4-46be-bc45-698200f08e54",
      "entityType" : "org.apache.nifi.flowfile.FlowFile",
      "entitySize" : 0,
      "previousEntitySize" : null,
      "updatedAttributes" : {
        "path" : "./",
        "filename" : "464535310815392",
        "uuid" : "ef12f52c-4ae4-46be-bc45-698200f08e54"
      },
      "previousAttributes" : { },
      "actorHostname" : "10.0.0.15",
      "contentURI" : "http://10.0.0.15:8080/nifi-api/provenance-events/13585491/content/output",
      "previousContentURI" : "http://10.0.0.15:8080/nifi-api/provenance-events/13585491/content/input",
      "parentIds" : [ ],
      "childIds" : [ ],
      "platform" : "nifi",
      "application" : "NiFi Flow",
      "transitUri" : null
    }
    {code}
    
    Notice timestamp millis.  Seems like we should let json and csv writing use the timestamp like it would be in an avro 'timestamp-millis' sort of way rather than a formatted date/time string.  We'd want to do this consistently on reading and writing.  The current default changes its representation i think and makes it a bit tougher to do math with.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1695: NIFI-3739: Added ConsumeKafkaRecord_0_10 and Publis...

Posted by joewitt <gi...@git.apache.org>.
Github user joewitt commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1695#discussion_r114150192
  
    --- Diff: nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java ---
    @@ -392,13 +394,27 @@ private void writeDemarcatedData(final ProcessSession session, final List<Consum
             bundleMap.put(topicPartition, tracker);
         }
     
    +    private void rollback(final TopicPartition topicPartition) {
    +        final OffsetAndMetadata offsetAndMetadata = kafkaConsumer.committed(topicPartition);
    +        final long offset = offsetAndMetadata.offset();
    +        kafkaConsumer.seek(topicPartition, offset);
    +    }
    +
         private void writeRecordData(final ProcessSession session, final List<ConsumerRecord<byte[], byte[]>> records, final TopicPartition topicPartition) {
             FlowFile flowFile = session.create();
             try {
                 final RecordSetWriter writer;
                 try {
                     writer = writerFactory.createWriter(logger, flowFile, new ByteArrayInputStream(new byte[0]));
                 } catch (final Exception e) {
    +                logger.error(
    +                    "Failed to obtain a Record Writer for serializing Kafka messages. This generally happens because the "
    +                        + "Record Writer cannot obtain the appropriate Schema, due to failure to connect to a remote Schema Registry "
    +                        + "or due to the Schema Access Strategy being dependent upon FlowFile Attributes that are not available. "
    +                        + "Will roll back the Kafka session.", e);
    +
    +                rollback(topicPartition);
    --- End diff --
    
    i think we probably need to have error handling on this as well in case we cannot talk back to kafka during this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1695: NIFI-3739: Added ConsumeKafkaRecord_0_10 and Publis...

Posted by joewitt <gi...@git.apache.org>.
Github user joewitt commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1695#discussion_r114151491
  
    --- Diff: nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java ---
    @@ -392,13 +394,27 @@ private void writeDemarcatedData(final ProcessSession session, final List<Consum
             bundleMap.put(topicPartition, tracker);
         }
     
    +    private void rollback(final TopicPartition topicPartition) {
    +        final OffsetAndMetadata offsetAndMetadata = kafkaConsumer.committed(topicPartition);
    --- End diff --
    
    instead of asking the kafka client where we were as far as committed offsets we should first check the uncommittedOffsets map and if nothing then ask the kafkaclient otherwise use the last uncommittedOffset for this topic/partition pair.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1695: NIFI-3739: Added ConsumeKafkaRecord_0_10 and PublishKafkaR...

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on the issue:

    https://github.com/apache/nifi/pull/1695
  
    @joewitt excellent! Thanks for such a diligent review & testing - you're certainly keeping me on my toes! ;-)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1695: NIFI-3739: Added ConsumeKafkaRecord_0_10 and PublishKafkaR...

Posted by joewitt <gi...@git.apache.org>.
Github user joewitt commented on the issue:

    https://github.com/apache/nifi/pull/1695
  
    @markap14 can you please squash at this point and rebase with master (now conflicts).  Thanks


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1695: NIFI-3739: Added ConsumeKafkaRecord_0_10 and Publis...

Posted by joewitt <gi...@git.apache.org>.
Github user joewitt commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1695#discussion_r114149213
  
    --- Diff: nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaRecord_0_10.java ---
    @@ -160,6 +163,11 @@
                 .name("success")
                 .description("FlowFiles received from Kafka.  Depending on demarcation strategy it is a flow file per message or a bundle of messages grouped by topic and partition.")
                 .build();
    +    static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
    --- End diff --
    
    @markap14 this is good but i think we'll need the metadata (topic,partition,offset) as that would obviously be helpful to folks troubleshooting such issues.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1695: NIFI-3739: Added ConsumeKafkaRecord_0_10 and PublishKafkaR...

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on the issue:

    https://github.com/apache/nifi/pull/1695
  
    @joewitt agreed on all fronts. Will push a new commit.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1695: NIFI-3739: Added ConsumeKafkaRecord_0_10 and PublishKafkaR...

Posted by joewitt <gi...@git.apache.org>.
Github user joewitt commented on the issue:

    https://github.com/apache/nifi/pull/1695
  
    ok i think we're there now.  Will merge shortly.  +1


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1695: NIFI-3739: Added ConsumeKafkaRecord_0_10 and PublishKafkaR...

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on the issue:

    https://github.com/apache/nifi/pull/1695
  
    @joewitt agreed. Pushed a new commit.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1695: NIFI-3739: Added ConsumeKafkaRecord_0_10 and Publis...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/nifi/pull/1695


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1695: NIFI-3739: Added ConsumeKafkaRecord_0_10 and PublishKafkaR...

Posted by joewitt <gi...@git.apache.org>.
Github user joewitt commented on the issue:

    https://github.com/apache/nifi/pull/1695
  
    can we update this message "ConsumeKafkaRecord_0_10[id=b28c1bea-015b-1000-db37-08c3c0beaeee] Failed to parse message from Kafka using the configured Record Reader. Will route message as its own FlowFile to the 'parse.failure' relationship: org.apache.nifi.schema.access.SchemaNotFoundException: Schema Encoding appears to be of an incompatible version. The latest known Protocol is Version 1 but the data was encoded with version 80
    "
    
    to indicate that either the encoded version is a given value or the data is not encoded in this form at all.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1695: NIFI-3739: Added ConsumeKafkaRecord_0_10 and Publis...

Posted by joewitt <gi...@git.apache.org>.
Github user joewitt commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1695#discussion_r114151657
  
    --- Diff: nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java ---
    @@ -408,53 +424,63 @@ private void writeRecordData(final ProcessSession session, final List<ConsumerRe
                 flowFile = session.write(flowFile, rawOut -> {
                     final Iterator<ConsumerRecord<byte[], byte[]>> itr = records.iterator();
     
    -                try (final OutputStream out = new BufferedOutputStream(rawOut)) {
    -                    final RecordSchema emptySchema = new SimpleRecordSchema(Collections.emptyList());
    -                    final RecordSet recordSet = new RecordSet() {
    -                        @Override
    -                        public RecordSchema getSchema() throws IOException {
    -                            return emptySchema;
    -                        }
    -
    -                        @Override
    -                        public Record next() throws IOException {
    -                            if (!itr.hasNext()) {
    -                                return null;
    -                            }
    +                final RecordSchema emptySchema = new SimpleRecordSchema(Collections.emptyList());
    +                final RecordSet recordSet = new RecordSet() {
    +                    @Override
    +                    public RecordSchema getSchema() throws IOException {
    +                        return emptySchema;
    +                    }
     
    +                    @Override
    +                    public Record next() throws IOException {
    +                        while (itr.hasNext()) {
                                 final ConsumerRecord<byte[], byte[]> consumerRecord = itr.next();
     
                                 final InputStream in = new ByteArrayInputStream(consumerRecord.value());
                                 try {
                                     final RecordReader reader = readerFactory.createRecordReader(ff, in, logger);
                                     final Record record = reader.nextRecord();
                                     return record;
    -                            } catch (final SchemaNotFoundException | MalformedRecordException e) {
    -                                throw new IOException(e);
    +                            } catch (final Exception e) {
    +                                FlowFile failureFlowFile = session.create();
    --- End diff --
    
    lets add the attributes we know for this message (topic/partition/offset)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---