*[KIP-305](https://cwiki.apache.org/confluence/display/KAFKA/KIP-305%3A+Add+Connect+primitive+number+converters) has been approved.*
Added converters and header converters for the primitive number types for which Kafka already had serializers and deserializers. All extend a common base class, `NumberConverter`, that encapsulates most of the shared functionality. Unit tests were added to check the basic functionality.
These classes are not used by any other Connect code, and must be explicitly used in Connect workers and connectors.
Author: Randall Hauch <rhauch@gmail.com>
Reviewers: Arjun Satish <wicknicks@users.noreply.github.com>, Magesh Nandakumar <magesh.n.kumar@gmail.com>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#5034 from rhauch/kafka-6913
Implementation of [KIP-174](https://cwiki.apache.org/confluence/display/KAFKA/KIP-174+-+Deprecate+and+remove+internal+converter+configs+in+WorkerConfig)
Configuration properties 'internal.key.converter' and 'internal.value.converter'
are deprecated, and default to org.apache.kafka.connect.json.JsonConverter.
Warnings are logged if values are specified for either, or if properties that
appear to configure instances of internal converters (i.e., ones prefixed with
either 'internal.key.converter.' or 'internal.value.converter.') are given.
The property 'schemas.enable' is also defaulted to false for internal
JsonConverter instances (both for keys and values) if it isn't specified.
Documentation and code have also been updated with deprecation notices and
annotations, respectively.
Unit tests have been updated in `PluginsTest` to account for the new defaults for `schemas.enable` for internal key/value converters, and to ensure that (for the time being), internal key/value converters are still configurable despite being deprecated.
Author: Chris Egerton <chrise@confluent.io>
Author: Ewen Cheslack-Postava <me@ewencp.org>
Reviewers: Randall Hauch <rhauch@gmail.com>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#4693 from C0urante/kafka-5540
Add the new stricter-timeout version of `poll` proposed in KIP-266.
The pre-existing variant `poll(long timeout)` would block indefinitely for metadata
updates if they were needed, then it would issue a fetch and poll for `timeout` ms
for new records. The initial indefinite metadata block caused applications to become
stuck when the brokers became unavailable. The existence of the timeout parameter
made the indefinite block especially unintuitive.
This PR adds `poll(Duration timeout)` with the semantics:
1. iff a metadata update is needed:
1. send (asynchronous) metadata requests
2. poll for metadata responses (counts against timeout)
- if no response within timeout, **return an empty collection immediately**
2. if there is fetch data available, **return it immediately**
3. if there is no fetch request in flight, send fetch requests
4. poll for fetch responses (counts against timeout)
- if no response within timeout, **return an empty collection** (leaving async fetch request for the next poll)
- if we get a response, **return the response**
The old method, `poll(long timeout)` is deprecated, but we do not change its semantics, so it remains:
1. iff a metadata update is needed:
1. send (asynchronous) metadata requests
2. poll for metadata responses *indefinitely until we get it*
2. if there is fetch data available, **return it immediately**
3. if there is no fetch request in flight, send fetch requests
4. poll for fetch responses (counts against timeout)
- if no response within timeout, **return an empty collection** (leaving async fetch request for the next poll)
- if we get a response, **return the response**
One notable usage is prohibited by the new `poll`: previously, you could call `poll(0)` to block for metadata updates, for example to initialize the client, supposedly without fetching records. Note, though, that this behavior is not according to any contract, and there is no guarantee that `poll(0)` won't return records the first time it's called. Therefore, it has always been unsafe to ignore the response.
Little back story on this. Was helping a user over email. This could be much easier to debug if we assume that the connector developer might not return valid configs. For example Intellij will generate a stub that returns a null. This was the case that inspired this JIRA.
Author: Jeremy Custenborder <jcustenborder@gmail.com>
Reviewers: Jason Gustafson <jason@confluent.io>, Randall Hauch <rhauch@gmail.com>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#3762 from jcustenborder/KAFKA-5807
We no longer need them since we now require Java 8.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Andras Beni <andrasbeni@cloudera.com>, Manikumar Reddy O <manikumar.reddy@gmail.com>, Dong Lin <lindong28@gmail.com>
Closes#5049 from ijuma/remove-base64
* Set --source, --target and --release to 1.8.
* Build Scala 2.12 by default.
* Remove some conditionals in the build file now that Java 8
is the minimum version.
* Bump the version of Jetty, Jersey and Checkstyle (the newer
versions require Java 8).
* Fixed issues uncovered by the new version if Checkstyle.
* A couple of minor updates to handle an incompatible source
change in the new version of Jetty.
* Add dependency to jersey-hk2 to fix failing tests caused
by Jersey upgrade.
* Update release script to use Java 8 and to take into account
that Scala 2.12 is now built by default.
* While we're at it, bump the version of Gradle, Gradle plugins,
ScalaLogging, JMH and apache directory api.
* Minor documentation updates including the readme and upgrade
notes. A number of Streams Java 7 examples can be removed
subsequently.
https://issues.apache.org/jira/browse/KAFKA-6685
Added Exception message in `WorkerSinkTask.convertMessages` to distinguish message Key from Value during deserialization to Kafka connect format.
*More detailed description of your change,
if necessary. The PR title and PR message become
the squashed commit message, so use a separate
comment to ping reviewers.*
*Summary of testing strategy (including rationale)
for the feature or bug fix. Unit and/or integration
tests are expected for any behaviour change and
system tests should be considered for larger changes.*
Author: Jagadesh Adireddi <adireddijagadesh@gmail.com>
Reviewers: Randall Hauch <rhauch@gmail.com>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#4765 from jadireddi/KAFKA-6685---log-message-should-distinguish-key-from-value
This is a change to improve resource cleanup for sink tasks and source tasks. Now `Task.stop()` is called from both `WorkerSinkTask.close()` and `WorkerSourceTask.close()`.
It is called from `WorkerXXXTask.close()` since this method is called in the `finally` block of `WorkerTask.run()`, and Connect developers use `stop()` to clean up resources.
Author: Robert Yokota <rayokota@gmail.com>
Reviewers: Randall Hauch <rhauch@gmail.com>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#5020 from rayokota/K6566-improve-connect-resource-cleanup
When AdminClient gets a NOT_CONTROLLER error, it should refresh its metadata and retry the request, rather than making the end-user deal with NotControllerException.
Move AdminClient's metadata management outside of NetworkClient and into AdminMetadataManager. This will make it easier to do more sophisticated metadata management in the future, such as implementing a NodeProvider which fetches the leaders for topics.
Rather than manipulating newCalls directly, the AdminClient service thread now drains it directly into pendingCalls. This minimizes the amount of locking we have to do, since pendingCalls is only accessed from the service thread.
This change makes adding a metric to a sensor idempotent.
That is, if the metric is already added to the sensor, the method
returns with success.
The current behavior is that any attempt to register a second metric
with the same name is an error.
Testing strategy: There is a new unit test covering this behavior
Reviewers: Guozhang Wang <guozhang@confluent.io>, Bill Bejeck <bill@confluent.io>
## Summary of the problem
When the `header.converter` is not specified in the worker config or the connector config, a bug in the `Plugins` test causes it to never instantiate the `HeaderConverter` instance, even though there is a default value.
This is a problem as soon as the connector deals with headers, either in records created by a source connector or in messages on the Kafka topics consumed by a sink connector. As soon as that happens, a NPE occurs.
A workaround is to explicitly set the `header.converter` configuration property, but this was added in AK 1.1 and thus means that upgrading to AK 1.1 will not be backward compatible and will require this configuration change.
## The Changes
The `Plugins.newHeaderConverter` methods were always returning null if the `header.converter` configuration value was not specified in the supplied connector or worker configuration. Thus, even though the `header.converter` property has a default, it was never being used.
The fix was to only check whether a `header.converter` property was specified when the connector configuration was being used, and if no such property exists in the connector configuration to return null. Then, when the worker configuration is being used, the method simply gets the `header.converter` value (or the default if no value was explicitly set).
Also, the ConnectorConfig had the same default value for the `header.converter` property as the WorkerConfig, but this resulted in very confusing log messages that implied the default header converter should be used even when the worker config specified the `header.converter` value. By removing the default, the log messages now make sense, and the Worker still properly instantiates the correct header converter.
Finally, updated comments and added log messages to make it more clear which converters are being used and how they are being converted.
## Testing
Several new unit tests for `Plugins.newHeaderConverter` were added to check the various behavior. Additionally, the runtime JAR with these changes was built and inserted into an AK 1.1 installation, and a source connector was manually tested with 8 different combinations of settings for the `header.converter` configuration:
1. default value
1. worker configuration has `header.converter` explicitly set to the default
1. worker configuration has `header.converter` set to a custom `HeaderConverter` implementation in the same plugin
1. worker configuration has `header.converter` set to a custom `HeaderConverter` implementation in a _different_ plugin
1. connector configuration has `header.converter` explicitly set to the default
1. connector configuration has `header.converter` set to a custom `HeaderConverter` implementation in the same plugin
1. connector configuration has `header.converter` set to a custom `HeaderConverter` implementation in a _different_ plugin
1. worker configuration has `header.converter` explicitly set to the default, and the connector configuration has `header.converter` set to a custom `HeaderConverter` implementation in a _different_ plugin
The worker created the correct `HeaderConverter` implementation with the correct configuration in all of these tests.
Finally, the default configuration was used with the aforementioned custom source connector that generated records with headers, and an S3 connector that consumes the records with headers (but didn't do anything with them). This test also passed.
Author: Randall Hauch <rhauch@gmail.com>
Reviewers: Ewen Cheslack-Postava <ewen@confluent.io>
Closes#4815 from rhauch/kafka-6728
Changed WorkerSinkTaskContext to only resume the consumer topic partitions when the connector/task is not in the paused state.
The context tracks the set of topic partitions that are explicitly paused/resumed by the connector, and when the WorkerSinkTask resumes the tasks it currently resumes all topic partitions *except* those that are still explicitly paused in the context. Therefore, the change above should result in the desired behavior.
Several debug statements were added to record when the context is called by the connector.
This can be backported to older releases, since this bug goes back to 0.10 or 0.9.
Author: Randall Hauch <rhauch@gmail.com>
Reviewers: Ewen Cheslack-Postava <ewen@confluent.io>
Closes#4716 from rhauch/kafka-6661
Any exception thrown by calls within a `main()` method are not logged unless explicitly done so. This change simply adds a try-catch block around most of the content of the distributed and standalone `main()` methods.
**NOTE: This should be backported to the `1.1` branch, and is currently a blocker for 1.1.**
The `connect_test.py::ConnectStandaloneFileTest.test_file_source_and_sink` system test is failing with the SASL configuration without a sufficient explanation. During the test, the Connect worker fails to start, but the Connect log contains no useful information. There are actual several things compounding to cause the failure and make it difficult to understand the problem.
First, the `tests/kafkatest/tests/connect/templates/connect_standalone.properties` is only adding in the broker's security configuration with the `producer.` and `consumer.` prefixes, but is not adding them with no prefix. The worker uses the AdminClient to connect to the broker to get the Kafka cluster ID and to manage the three internal topics, and the AdminClient is configured via top-level properties. Because the SASL test requires the clients all connect using SASL, the lack of broker security configs means the AdminClient was attempting and failing to connect to the broker. This is corrected by adding the broker's security configuration to the Connect worker configuration file at the top-level. (This was already being done in the `connect_distributed.properties` file.)
Second, the default `request.timeout.ms` for the AdminClient (and the other clients) is 120 seconds, so the AdminClient was retrying for 120 seconds before it would give up and thrown an error. However, the test was only waiting for 60 seconds before determining that the service failed to start. This can be corrected by setting `request.timeout.ms=10000` in the Connect distributed and standalone worker configurations.
Third, the Connect workers were recently changed to lookup the Kafka cluster ID before it started the herder. This is unlike the older uses of the AdminClient to find and manage the internal topics, where failure to connect was not necessarily logged correctly but nevertheless still skipped over, relying upon broker auto-topic creation to create the internal topics. (This may be why the test did not fail prior to the recent change to always require a successful AdminClient connection.) Although the worker never got this far in its startup process, the fact that we missed such an error since the prior releases means that failure to connect with the AdminClient was not being properly reported.
The `ConnectStandaloneFileTest.test_file_source_and_sink` system tests were run locally prior to this fix, and they failed as with the nightlies. Once these fixes were made, the locally run system tests passed.
Author: Randall Hauch <rhauch@gmail.com>
Reviewers: Konstantine Karantasis <konstantine@confluent.io>, Ewen Cheslack-Postava <me@ewencp.org>
Closes#4610 from rhauch/kafka-6577-trunk
Enabling scans for super types in reflections is required in order to discover Connect plugins.
Reviewers: Randall Hauch <rhauch@gmail.com>, Jason Gustafson <jason@confluent.io>
Sending the response code of an http request issued via `RestClient` in Connect to stdout seems like a unconventional choice.
This PR redirects the responds code with a message in the logs at DEBUG level (usually the same level as the one that the caller of `RestClient.httpRequest` uses.
This fix will also fix system tests that broke by outputting this response code to stdout.
Author: Konstantine Karantasis <konstantine@confluent.io>
Reviewers: Randall Hauch <rhauch@gmail.com>, Damian Guy <damian.guy@gmail.com>
Closes#4591 from kkonstantine/MINOR-Redirect-response-code-in-Connect-RestClient-to-logs-instead-of-stdout
https://github.com/apache/kafka/pull/4356 added `batch.size` config property to `FileStreamSourceConnector` but the property was added as required without a default in config definition (`ConfigDef`). This results in validation error during connector startup.
Unit tests were added for both `FileStreamSourceConnector` and `FileStreamSinkConnector` to avoid such issues in the future.
Reviewers: Randall Hauch <rhauch@gmail.com>, Jason Gustafson <jason@confluent.io>
This is a small change to parallelize plugin scanning. This may help in some environments where otherwise plugin scanning is slow.
Author: Robert Yokota <rayokota@gmail.com>
Reviewers: Konstantine Karantasis <konstantine@confluent.io>, Randall Hauch <rhauch@gmail.com>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#4561 from rayokota/K6503-improve-plugin-scanning
JsonConverter should use object equality rather than reference equality in `convertToJson`.
Reviewers: Bartlomiej Tartanus <bartektartanus@gmail.com>, Randall Hauch <rhauch@gmail.com>, Jason Gustafson <jason@confluent.io>
Corrected the parsing of invalid list values. A list can only be parsed if it contains elements that have a common type, and a map can only be parsed if it contains keys with a common type and values with a common type.
Reviewers: Arjun Satish <arjun@confluent.io>, Magesh Nandakumar <magesh.n.kumar@gmail.com>, Konstantine Karantasis <konstantine@confluent.io>, Jason Gustafson <jason@confluent.io>
Changed call to use the overload of ConnectSchema.validateValue() method with the field name passed in. Ensure that field in put call is not null.
Reviewers: Randall Hauch <rhauch@gmail.com>, Jason Gustafson <jason@confluent.io>
The commits for KIP-145 (KAFKA-5142) changed how the Connect workers instantiate and configure the Converters, and also added the ability to do the same for the new HeaderConverters. However, the last few commits removed the default value for the `converter.type` property for Converters and HeaderConverters, and this broke how the internal converters were being created.
This change corrects the behavior so that the `converter.type` property is always set by the worker (or by the Plugins class), which means the existing Converter implementations will not have to do this. The built-in JsonConverter, ByteArrayConverter, and StringConverter also implement HeaderConverter which implements Configurable, but the Worker and Plugins methods do not yet use the `Configurable.configure(Map)` method and instead still use the `Converter.configure(Map,boolean)`.
Several tests were modified, and a new PluginsTest was added to verify the new behavior in Plugins for instantiating and configuring the Converter and HeaderConverter instances.
Author: Randall Hauch <rhauch@gmail.com>
Reviewers: Konstantine Karantasis <konstantine@confluent.io>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#4512 from rhauch/kafka-6513
This change ensures that when sensors are created, they are unique to the metric group associated with the task that created them. Previously the sensors were being shared between task metric groups, causing incorrect metrics.
Reviewers: Randall Hauch <rhauch@gmail.com>, Jason Gustafson <jason@confluent.io>
KAFKA-3073 added topic regex support for sink connectors. The addition requires that you only specify one of topics or topics.regex settings. This is being validated in one place, but not during submission of connectors. This PR adds validation at `AbstractHerder.validateConnectorConfig` and `WorkerConnector.initialize`.
This adds a test of the new behavior to `AbstractHerderTest`.
Author: Jeff Klukas <jeff@klukas.net>
Reviewers: Randall Hauch <rhauch@gmail.com>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#4251 from jklukas/connect-topics-validation
Submitting a fail safe fix for rare IOExceptions on symbolic links.
The fix is submitted without a test case since it does seem easy to reproduce such type of failures (just having a broken symbolic link does not reproduce the issue) and it's considered pretty low risk.
If accepted, needs to be ported at least to 1.0, if not 0.11
Author: Konstantine Karantasis <konstantine@confluent.io>
Reviewers: Randall Hauch <rhauch@gmail.com>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#4481 from kkonstantine/KAFKA-6288-Broken-symlink-interrupts-scanning-the-plugin-path
**[KIP-145](https://cwiki.apache.org/confluence/display/KAFKA/KIP-145+-+Expose+Record+Headers+in+Kafka+Connect) has been accepted, and this PR implements KIP-145 except without the SMTs.**
Changed the Connect API and runtime to support message headers as described in [KIP-145](https://cwiki.apache.org/confluence/display/KAFKA/KIP-145+-+Expose+Record+Headers+in+Kafka+Connect).
The new `Header` interface defines an immutable representation of a Kafka header (key-value pair) with support for the Connect value types and schemas. This interface provides methods for easily converting between many of the built-in primitive, structured, and logical data types.
The new `Headers` interface defines an ordered collection of headers and is used to track all headers associated with a `ConnectRecord` (and thus `SourceRecord` and `SinkRecord`). This does allow multiple headers with the same key. The `Headers` contains methods for adding, removing, finding, and modifying headers. Convenience methods allow connectors and transforms to easily use and modify the headers for a record.
A new `HeaderConverter` interface is also defined to enable the Connect runtime framework to be able to serialize and deserialize headers between the in-memory representation and Kafka’s byte[] representation. A new `SimpleHeaderConverter` implementation has been added, and this serializes to strings and deserializes by inferring the schemas (`Struct` header values are serialized without the schemas, so they can only be deserialized as `Map` instances without a schema.) The `StringConverter`, `JsonConverter`, and `ByteArrayConverter` have all been extended to also be `HeaderConverter` implementations. Each connector can be configured with a different header converter, although by default the `SimpleHeaderConverter` is used to serialize header values as strings without schemas.
Unit and integration tests are added for `ConnectHeader` and `ConnectHeaders`, the two implementation classes for headers. Additional test methods are added for the methods added to the `Converter` implementations. Finally, the `ConnectRecord` object is already used heavily, so only limited tests need to be added while quite a few of the existing tests already cover the changes.
Author: Randall Hauch <rhauch@gmail.com>
Reviewers: Arjun Satish <arjun@confluent.io>, Ted Yu <yuzhihong@gmail.com>, Magesh Nandakumar <magesh.n.kumar@gmail.com>, Konstantine Karantasis <konstantine@confluent.io>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#4319 from rhauch/kafka-5142-b
…to check for empty connector name and illegal characters in connector name. This also fixes KAFKA-4938 by removing the check for slashes in connector name from ConnectorsResource.
Author: Ewen Cheslack-Postava <me@ewencp.org>
Author: Soenke Liebau <soenke.liebau@opencore.com>
Reviewers: Gwen Shapira <cshapi@gmail.com>, Viktor Somogyi <viktor.somogyi@cloudera.com>, Randall Hauch <rhauch@gmail.com>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#2755 from soenkeliebau/KAFKA-4930
This PR implements the JIRA issue [KAFKA-4029: SSL support for Connect REST API](https://issues.apache.org/jira/browse/KAFKA-4029) / [KIP-208](https://cwiki.apache.org/confluence/display/KAFKA/KIP-208%3A+Add+SSL+support+to+Kafka+Connect+REST+interface).
Summary of the main changes:
- Jetty `HttpClient` is used as HTTP client instead of the one shipped with Java. That allows to keep the SSL configuration for Server and Client be in single place (both use the Jetty `SslContextFactory`). It also has much richer configuration than the JDK client (it is easier to configure things such as supported cipher suites etc.).
- The `RestServer` class has been broker into 3 parts. `RestServer` contains the server it self. `RestClient` contains the HTTP client used for forwarding requests etc. and `SSLUtils` contain some helper classes for configuring SSL. One of the reasons for this was Findbugs complaining about the class complexity.
- A new method `valuesWithPrefixAllOrNothing` has been added to `AbstractConfig` to make it easier to handle the situation that we want to use either only the prefixed SSL options or only the non-prefixed. But not mixed them.
Author: Jakub Scholz <www@scholzj.com>
Reviewers: Ewen Cheslack-Postava <ewen@confluent.io>
Closes#4429 from scholzj/kip-208
Exclusion for packages that need not be loaded in isolation needs to be extended to all the `org.apache.kafka` packages (that do not belong to transforms and the other whitelisted packages). Most notably, this refers to any classes in `kafka-clients` package.
Reviewers: Randall Hauch <rhauch@gmail.com>, Jason Gustafson <jason@confluent.io>
Adds alphanumeric ordering of dependencies as they added to a Connect plugin's class loader path.
This makes the layout of the dependencies consistent across systems and deployments. Dependencies should still, in principle, not include conflicts and ideally order should not matter.
Reviewers: Randall Hauch <rhauch@gmail.com>, Jason Gustafson <jason@confluent.io>
`loadClass` needs to be synchronized to protect subsequent calls to `defineClass`.
Details in the javadoc of this PR as well as here too: https://docs.oracle.com/javase/7/docs/technotes/guides/lang/cl-mt.html
/cc ewencp rhauch
Author: Konstantine Karantasis <konstantine@confluent.io>
Reviewers: Ewen Cheslack-Postava <ewen@confluent.io>
Closes#4428 from kkonstantine/KAFKA-6277-Make-loadClass-thread-safe-for-class-loaders-of-Connect-plugins
Making clear that implementations of poll() shouldn't block indefinitely in order to allow the task instance to transition to PAUSED state.
Reviewers: Randall Hauch <rhauch@gmail.com>, Jason Gustafson <jason@confluent.io>
When using Kafka Connect with a cluster that doesn't allow the user to
create topics (due to ACL configuration), Connect fails when trying to
create its internal topics even if these topics already exist. This is
incorrect behavior according to the documentation, which mentions that
R/W access should be enough.
This happens specifically when using Aiven Kafka, which does not permit
creation of topics via the Kafka Admin Client API.
The patch ignores the returned error, similar to the behavior for older
brokers that don't support the API.
* Implement MockAdminClient.deleteTopics
* Use MockAdminClient instead of MockKafkaAdminClientEnv in StreamsResetterTest
* Rename MockKafkaAdminClientEnv to AdminClientUnitTestEnv
* Use MockAdminClient instead of MockKafkaAdminClientEnv in TopicAdminTest
* Rename KafkaAdminClient to AdminClientUnitTestEnv in KafkaAdminClientTest.java
* Migrate StreamThreadTest to MockAdminClient
* Fix style errors
* Address review comments
* Fix MockAdminClient call
Reviewers: Matthias J. Sax <matthias@confluent.io>, Konstantine Karantasis <konstantine@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
We are closing the metricGroups created in a Worker, Source task and Sink task before populating them with new metrics. This helps in cases where an Exception is thrown when previously created groups were not cleaned up correctly.
Signed-off-by: Arjun Satish <arjunconfluent.io>
Author: Arjun Satish <arjun@confluent.io>
Reviewers: Randall Hauch <rhauch@gmail.com>, Konstantine Karantasis <konstantine@confluent.io>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#4397 from wicknicks/KAFKA-6252
When the source file of `FileStreamSource` is a large file, `FileStreamSourceTask.poll()` will result in OOM. This pull request added `batch.size` parameter which can restrict the poll size.
*More detailed description of your change,
if necessary. The PR title and PR message become
the squashed commit message, so use a separate
comment to ping reviewers.*
*Summary of testing strategy (including rationale)
for the feature or bug fix. Unit and/or integration
tests are expected for any behaviour change and
system tests should be considered for larger changes.*
Author: Study <ph.study@gmail.com>
Reviewers: Ewen Cheslack-Postava <ewen@confluent.io>
Closes#4356 from phstudy/KAFKA-4335
This changes the Struct's equals and hashCode method to use Arrays#deepEquals and Arrays#deepHashCode, respectively. This resolves a problem where two structs with values of type byte[] would not be considered equal even though the byte arrays' contents are equal. By using deepEquals, the byte arrays' contents are compared instead of ther identity.
Since this changes the behavior of the equals method for byte array values, the behavior of hashCode must change alongside it to ensure the methods still fulfill the general contract of "equal objects must have equal hashCodes".
Test rationale:
All existing unit tests for equals were untouched and continue to work. A new test method was added to verify the behavior of equals and hashCode for Struct instances that contain a byte array value. I verify the reflixivity and transitivity of equals as well as the fact that equal Structs have equal hashCodes
and not-equal structs do not have equal hashCodes.
Author: Tobias Gies <tobias.gies@trivago.com>
Author: Tobias Gies <tobias@tobiasgies.de>
Reviewers: Randall Hauch <rhauch@gmail.com>, Jason Gustafson <jason@confluent.io>
Closes#4293 from tobiasgies/feature/kafka-6308-deepequals
Author: Colin P. Mccabe <cmccabe@confluent.io>
Reviewers: Randall Hauch <rhauch@gmail.com>, Jason Gustafson <jason@confluent.io>
Closes#4105 from cmccabe/KAFKA-6102
…from config to own function and added check to create connector call.
Author: Soenke Liebau <soenke.liebau@opencore.com>
Reviewers: Ewen Cheslack-Postava <ewen@confluent.io>
Closes#4230 from soenkeliebau/KAFKA-5563