Status
Current state: Under Discussion
Discussion thread: here [Change the link from the KIP proposal email archive to your own email thread]
JIRA: KAFKA-7402 [Change the link from KAFKA-1 to your own ticket]
Motivation
Quoting John John Roesler:
"Various components in Streams have close methods but do not implement AutoCloseable. This means that they can't be used in try-with-resources blocks.
Remedying that would simplify our tests and make life easier for users as well.
KafkaStreams itself is a notable example of this, but we can take the opportunity to look for other components that make sense as AutoCloseable as well."
From AutoCloseable's Javadoc:
An object that may hold resources (such as file or socket handles) until it is closed. The {@link #close()} method of an {@code AutoCloseable} object is called automatically when exiting a {@code try}-with-resources block for which the object has been declared in the resource specification header. This construction ensures prompt release, avoiding resource exhaustion exceptions and errors that may otherwise occur.
Public Interfaces
By going over the project, here is a list that I found which can implement AutoCloseable. Suggestions are welcome.
- org.apache.kafka.streams.KafkaStreams
- org.apache.kafka.streams.processor.internals.RecordCollectorImpl
- org.apache.kafka.connect.runtime.ConnectMetrics.MetricGroup
- org.apache.kafka.connect.transforms.TimestampRouter
- org.apache.kafka.tools.VerifiableProducer
Proposed Changes
Here are the proposed changes:
org.apache.kafka.streams.KafkaStreams
public class KafkaStreams implements AutoCloseable{ ... public void close() ... }
org.apache.kafka.streams.processor.internals.RecordCollectorImpl
public class RecordCollectorImpl implements RecordCollector, AutoCloseable { ... }
org.apache.kafka.connect.runtime.ConnectMetrics.MetricGroup
public class MetricGroup implements AutoCloseable { ... }
org.apache.kafka.connect.transforms.TimestampRouter
public class TimestampRouter<R extends ConnectRecord<R>> implements Transformation<R>, AutoCloseable { ... }
org.apache.kafka.tools.VerifiableProducer
public class VerifiableProducer implements Closeable { ... }
Compatibility, Deprecation, and Migration Plan
- N/A