GitHub user StephanEwen opened a pull request:

    https://github.com/apache/flink/pull/1017

    [FLINK-2462] [streaming] Major cleanup of streaming task structure

    This pull request addresses exception handling, code duplication, and 
missed resource cleanups in the streaming operators.
    
    I mixed multiple issues in this pull request, which would have been better 
separated, but many were recognized in the rework, and it was tricky to pull 
the fixes apart.
    
    **NOTE** I have not managed to adjust all tests, yet, but I wanted to open 
this early for feedback.
    
    ## Exception handling
    
    The exceptions are no longer logged by the operators themselves. Operators 
perform only cleanup in reaction to exceptions.
    
    Exceptions are reported only the the root Task object, which knows whether 
this is the first failure-causing exception (root cause), or is a subsequent 
exception, or whether the task was actually canceled already. In the later 
case, exceptions are ignored, because many cancellations lead to meaningless 
exceptions.
    
    Added more exception in signatures, less exception wrapping where not needed
    
    ## Unified setup / teardown structure in streaming tasks
    
    Core resource acquisition/release logic is in `StreamTask`, reducing code 
duplication.
    Subtasks (e.g., `OneInputStreamTask`, `IterationTailStreamTask`) implement 
slim methods for certain parts of the life cycle. The `OneInputStreamTask` 
becomes as simple as this
    
    ```java
    public void init() throws Exception {
        TypeSerializer<IN> inSerializer = 
configuration.getTypeSerializerIn1(getUserCodeClassLoader());
        InputGate[] inputGates = getEnvironment().getAllInputGates();
        inputProcessor = new StreamInputProcessor<IN>(inputGates, inSerializer,
                        getCheckpointBarrierListener(), 
                        configuration.getCheckpointMode(),
                        getEnvironment().getIOManager(),
                        getExecutionConfig().areTimestampsEnabled());
    
        // make sure that stream tasks report their I/O statistics
        AccumulatorRegistry registry = 
getEnvironment().getAccumulatorRegistry();
        AccumulatorRegistry.Reporter reporter = registry.getReadWriteReporter();
        inputProcessor.setReporter(reporter);
    }
    
    protected void run() throws Exception {
        while (running && inputProcessor.processInput(streamOperator));
    }
    
    protected void cleanup() throws Exception {
        inputProcessor.cleanup();
    }
    
    protected void cancelTask() {
        running = false;
    }
    ```
    Guaranteed cleanup of output buffer and input buffer resources (formerly 
missed when other exceptions where encountered).
    
    Unified `StreamRecordWriter` and `RecordWriter` usage.
    
    ## Cleanup in the StreamSource
    
    Fix mixup in instantiation of source contexts in the stream source task
    
    Auto watermark generators correctly shut down their interval scheduler
    
    ## General
    
    Improve use of generics, got rid of many raw types

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

    $ git pull https://github.com/StephanEwen/incubator-flink stream_cleanup

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

    https://github.com/apache/flink/pull/1017.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 #1017
    
----
commit 68efed0a3b4184980de956bd57ba301569adac86
Author: Stephan Ewen <se...@apache.org>
Date:   2015-08-14T21:32:35Z

    [FLINK-2462] [streaming] Major cleanup of operator structure for exception 
handling and code simplication
    
      - The exceptions are no longer logged by the operators themselves.
        Operators perform only cleanup in reaction to exceptions.
        Exceptions are reported only the the root Task object, which knows 
whether this is the first
        failure-causing exception (root cause), or is a subsequent exception, 
or whether the task was
        actually canceled already. In the later case, exceptions are ignored, 
because many
        cancellations lead to meaningless exceptions.
    
      - more exception in signatures, less wrapping where not needed
    
      - Core resource acquisition/release logic is in one streaming task, 
reducing code duplication
    
      - Guaranteed cleanup of output buffer and input buffer resources 
(formerly missed when other exceptions where encountered)
    
      - Fix mixup in instantiation of source contexts in the stream source task
    
      - Auto watermark generators correctly shut down their interval scheduler
    
      - Improve use of generics, got rid of many raw types

----


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

Reply via email to