You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@nifi.apache.org by joewitt <gi...@git.apache.org> on 2016/04/22 18:27:23 UTC

[GitHub] nifi pull request: NIFI-1296, NIFI-1680, NIFI-1764 implemented new...

Github user joewitt commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/366#discussion_r60766260
  
    --- Diff: nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/AbstractKafkaProcessor.java ---
    @@ -0,0 +1,125 @@
    +/*
    + * 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.nifi.processors.kafka;
    +
    +import java.io.Closeable;
    +import java.util.concurrent.atomic.AtomicInteger;
    +
    +import org.apache.kafka.clients.consumer.KafkaConsumer;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.ProcessSessionFactory;
    +import org.apache.nifi.processor.Processor;
    +import org.apache.nifi.processor.exception.ProcessException;
    +
    +/**
    + * Base class for {@link Processor}s to publish and consume messages from Kafka
    + *
    + * @see PutKafka
    + */
    +abstract class AbstractKafkaProcessor<T extends Closeable> extends AbstractSessionFactoryProcessor {
    +
    +
    +    private volatile boolean acceptTask = true;
    +
    +    private final AtomicInteger taskCounter = new AtomicInteger();
    +
    +
    +    /**
    +     * @see KafkaPublisher
    +     */
    +    volatile T kafkaResource;
    +
    +    /**
    +     *
    +     */
    +    @Override
    +    public final void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) throws ProcessException {
    +        final ProcessSession session = sessionFactory.createSession();
    +        if (this.acceptTask) {
    +            try {
    +                this.taskCounter.incrementAndGet();
    +                if (this.kafkaResource == null) {
    +                    synchronized (this) {
    +                        if (this.kafkaResource == null) {
    +                            this.kafkaResource = this.buildKafkaResource(context, session);
    +                        }
    +                    }
    +                }
    +
    +                this.rendezvousWithKafka(context, session);
    +                session.commit();
    +            } catch (Throwable e) {
    +                this.acceptTask = false;
    +                this.getLogger().error("{} failed to process due to {}; rolling back session", new Object[] { this, e });
    +                session.rollback(true);
    +            } finally {
    +                this.resetProcessorIfNecessary();
    +            }
    +        } else {
    +            context.yield();
    +        }
    +    }
    +    /**
    +     * Resets the processor to initial state if necessary. The necessary state
    +     * means there are no active task which could only happen if currently
    +     * executing tasks are given a chance to finish while no new tasks are
    +     * accepted (see {@link #acceptTask}
    +     */
    +    private boolean resetProcessorIfNecessary() {
    +        boolean reset = this.taskCounter.decrementAndGet() == 0 && !this.acceptTask;
    +        if (reset) {
    +            this.close();
    +            this.acceptTask = true;
    +        }
    +        return reset;
    +    }
    +
    +    /**
    +     * Will call {@link Closeable#close()} on the target resource after which
    +     * the target resource will be set to null
    +     *
    +     * @see KafkaPublisher
    +     */
    +    @OnStopped
    +    public void close() {
    --- End diff --
    
    @olegz could we reduce the scope here?  The concern I have is that technically this is a check-then-modify scenario and it could be called by both internal processor threads executing resetProcessorIfNecessary and whenever the framework is calling onStopped.  Now, by the way the framework works onStopped should only be called once there are no threads left so technically this should be legit.  Just would not want someone to also call this from other places in the code if we can avoid it.  So recommend we reduce scope if possible and in either case document the fact that this should not be called by others.  My comments are critical as I think practically speaking this is solid but there is a tiny window of concern..


---
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.
---