Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

Table of Contents

Status

Current stateUnder DiscussionAccepted (2.2.0)

Discussion thread: Link 

JIRA: KAFKA-7402

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.

...

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

So these classes can be used in a try-with-resources Statement after implementing AutoCloseable.

...

  1. org.apache.kafka.streams.KafkaStreams
  2. org.apache.kafka.streams.processor.internals.RecordCollector
  3. org.apache.kafka.connect.runtime.ConnectMetrics.MetricGroup
  4. org.apache.kafka.connect.transforms.TimestampRouter
  5. org.apache.kafka.tools.VerifiableProducer

...

  1. org.apache.kafka.connect.runtime.WorkerConnector.ConnectorMetricsGroup
  2. org.apache.kafka.common.record.MemoryRecordsBuilder
  3. org.apache.kafka.common.metrics.MetricsReporter
  4. org.apache.kafka.common.security.oauthbearer.internals.expiring.ExpiringCredentialRefreshingLogin
  5. org.apache.kafka.common.network.KafkaChannel
  6. org.apache.kafka.clients.consumer.ConsumerInterceptor
  7. org.apache.kafka.common.network.Selector.SelectorMetrics
  8. org.apache.kafka.clients.consumer.internals.AbstractCoordinator.HeartbeatThread

Proposed Changes

Here are some examples for the proposed changesan example of how it would look like:

org.apache.kafka.streams.KafkaStreams

Code Block

public class KafkaStreams implements AutoCloseable{
...
	public void close()
...
}

org.apache.kafka.streams.processor.internals.RecordCollector

Code Block
public interface RecordCollector extends AutoCloseable {
...
}

org.apache.kafka.connect.runtime.ConnectMetrics.MetricGroup

Code Block
public class MetricGroup implements AutoCloseable {
...
}

org.apache.kafka.connect.transforms.TimestampRouter

Code Block
public class TimestampRouter<R extends ConnectRecord<R>> implements Transformation<R>, AutoCloseable {
...
}

org.apache.kafka.tools.VerifiableProducer

Code Block
public class VerifiableProducer implements AutoCloseable {
...
}

For Chia-ping's list, the changes are very similar ( ...implements AutoCloseable). Specific changes can be added to this KIP if requested by reviewers.

Compatibility, Deprecation, and Migration Plan


Compatibility, Deprecation, and Migration Plan

  • This change will not have backward compatibility issues. It is legal to implement this AutoClosable interface without declaring "throws Exception" on close(), so the close() method signature won't be changed.  


Rejected Alternatives

  • We decided not to choose Closeable(). Closeable extends AutoCloseable, and AutoCloseable throws a broader exception than Closeable,  so we think that AutoCloseable is a more generic and a more compatible option.N/A